Skip to content

Commit

Permalink
sync request/response for sdk in rocketmq-connector (#634)
Browse files Browse the repository at this point in the history
fix sync request/response for sdk in rocketmq-connector
  • Loading branch information
xwm1992 authored Dec 8, 2021
1 parent 3b82ce6 commit 46471ed
Show file tree
Hide file tree
Showing 14 changed files with 253 additions and 25 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,10 +35,13 @@
import org.apache.eventmesh.connector.rocketmq.utils.OMSUtil;
import org.apache.eventmesh.connector.rocketmq.utils.CloudEventUtils;

import org.apache.commons.lang3.StringUtils;
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageConcurrentlyService;
import org.apache.rocketmq.client.impl.consumer.ConsumeMessageService;
import org.apache.rocketmq.client.utils.MessageUtil;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.common.protocol.heartbeat.MessageModel;
import org.apache.rocketmq.remoting.protocol.LanguageCode;
Expand All @@ -51,6 +54,7 @@
import java.util.concurrent.atomic.AtomicBoolean;

import io.cloudevents.CloudEvent;
import io.cloudevents.core.builder.CloudEventBuilder;

public class PushConsumerImpl {
private final DefaultMQPushConsumer rocketmqPushConsumer;
Expand Down Expand Up @@ -178,9 +182,28 @@ public EventMeshConsumeConcurrentlyStatus handleMessage(MessageExt msg,
msg.putUserProperty(Constants.PROPERTY_MESSAGE_STORE_TIMESTAMP,
String.valueOf(msg.getStoreTimestamp()));

//for rr request/reply
String cluster = msg.getProperty(MessageConst.PROPERTY_CLUSTER);
String replyClient = msg.getProperty(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT);
String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);

CloudEvent cloudEvent =
RocketMQMessageFactory.createReader(CloudEventUtils.msgConvert(msg)).toEvent();

CloudEventBuilder cloudEventBuilder;
if (StringUtils.isNotEmpty(cluster)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("cluster", cluster);
cloudEvent = cloudEventBuilder.build();
}
if (StringUtils.isNotEmpty(replyClient)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("replytoclient", replyClient);
cloudEvent = cloudEventBuilder.build();
}
if (StringUtils.isNotEmpty(correlationId)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("correlationid", correlationId);
cloudEvent = cloudEventBuilder.build();
}

EventListener listener = PushConsumerImpl.this.subscribeTable.get(msg.getTopic());

if (listener == null) {
Expand Down Expand Up @@ -236,9 +259,28 @@ public EventMeshConsumeConcurrentlyStatus handleMessage(MessageExt msg,
msg.putUserProperty(EventMeshConstants.STORE_TIMESTAMP,
String.valueOf(msg.getStoreTimestamp()));

//for rr request/reply
String cluster = msg.getProperty(MessageConst.PROPERTY_CLUSTER);
String replyClient = msg.getProperty(MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT);
String correlationId = msg.getProperty(MessageConst.PROPERTY_CORRELATION_ID);

CloudEvent cloudEvent =
RocketMQMessageFactory.createReader(CloudEventUtils.msgConvert(msg)).toEvent();

CloudEventBuilder cloudEventBuilder;
if (StringUtils.isNotEmpty(cluster)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("cluster", cluster);
cloudEvent = cloudEventBuilder.build();
}
if (StringUtils.isNotEmpty(replyClient)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("replytoclient", replyClient);
cloudEvent = cloudEventBuilder.build();
}
if (StringUtils.isNotEmpty(correlationId)) {
cloudEventBuilder = CloudEventBuilder.from(cloudEvent).withExtension("correlationid", correlationId);
cloudEvent = cloudEventBuilder.build();
}

EventListener listener = PushConsumerImpl.this.subscribeTable.get(msg.getTopic());

if (listener == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,13 @@
import org.apache.eventmesh.connector.rocketmq.utils.OMSUtil;
import org.apache.eventmesh.connector.rocketmq.utils.CloudEventUtils;

import org.apache.commons.lang3.StringUtils;
import org.apache.rocketmq.client.exception.MQBrokerException;
import org.apache.rocketmq.client.exception.MQClientException;
import org.apache.rocketmq.client.producer.RequestCallback;
import org.apache.rocketmq.common.MixAll;
import org.apache.rocketmq.common.message.Message;
import org.apache.rocketmq.common.message.MessageAccessor;
import org.apache.rocketmq.common.message.MessageClientIDSetter;
import org.apache.rocketmq.common.message.MessageConst;
import org.apache.rocketmq.common.message.MessageExt;
Expand Down Expand Up @@ -130,7 +132,17 @@ public boolean reply(final CloudEvent cloudEvent, final SendCallback sendCallbac
this.checkProducerServiceState(this.rocketmqProducer.getDefaultMQProducerImpl());
org.apache.rocketmq.common.message.Message msg =
RocketMQMessageFactory.createWriter(cloudEvent.getSubject()).writeBinary(cloudEvent);
msg.putUserProperty(MessageConst.PROPERTY_MESSAGE_TYPE, MixAll.REPLY_MESSAGE_FLAG);
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_TYPE, MixAll.REPLY_MESSAGE_FLAG);
if (StringUtils.isNotEmpty(cloudEvent.getExtension("cluster").toString())) {
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_CLUSTER, cloudEvent.getExtension("cluster").toString());
}
if (StringUtils.isNotEmpty(cloudEvent.getExtension("replytoclient").toString())) {
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_MESSAGE_REPLY_TO_CLIENT, cloudEvent.getExtension("replytoclient").toString());
}
if (StringUtils.isNotEmpty(cloudEvent.getExtension("correlationid").toString())) {
MessageAccessor.putProperty(msg, MessageConst.PROPERTY_CORRELATION_ID, cloudEvent.getExtension("correlationid").toString());
}

try {
this.rocketmqProducer.send(msg, this.sendCallbackConvert(msg, sendCallback));
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public static Message msgConvert(MessageExt rmqMsg) {
}

//
MessageAccessor.putProperty(message,buildCloudEventPropertyKey(Constants.PROPERTY_MESSAGE_BORN_HOST),
MessageAccessor.putProperty(message, buildCloudEventPropertyKey(Constants.PROPERTY_MESSAGE_BORN_HOST),
String.valueOf(rmqMsg.getBornHost()));
MessageAccessor.putProperty(message, buildCloudEventPropertyKey(Constants.PROPERTY_MESSAGE_BORN_TIMESTAMP),
String.valueOf(rmqMsg.getBornTimestamp()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ public class OMSUtil {
* @return a unique instance name
*/
public static String buildInstanceName() {
return Integer.toString(UtilAll.getPid()) + "%OpenMessaging" + "%" + System.nanoTime();
return Integer.toString(UtilAll.getPid()) + "%EventMesh" + "%" + System.nanoTime();
}

// public static org.apache.rocketmq.common.message.Message msgConvert(Message omsMessage) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
import org.mockito.junit.MockitoJUnitRunner;

import io.cloudevents.CloudEvent;
import io.openmessaging.api.OMSBuiltinKeys;

@RunWith(MockitoJUnitRunner.class)
public class PushConsumerImplTest {
Expand All @@ -53,8 +52,8 @@ public class PushConsumerImplTest {
@Before
public void before() throws Exception {
Properties consumerProp = new Properties();
consumerProp.setProperty(OMSBuiltinKeys.DRIVER_IMPL,
"org.apache.eventmesh.connector.rocketmq.MessagingAccessPointImpl");
// consumerProp.setProperty(OMSBuiltinKeys.DRIVER_IMPL,
// "org.apache.eventmesh.connector.rocketmq.MessagingAccessPointImpl");
consumerProp.setProperty("access_points", "IP1:9876,IP2:9876");
//final MessagingAccessPoint messagingAccessPoint = OMS.builder().build(consumerProp);//.endpoint("oms:rocketmq://IP1:9876,IP2:9876/namespace").build(config);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@

import io.cloudevents.CloudEvent;
import io.cloudevents.core.builder.CloudEventBuilder;
import io.openmessaging.api.OMSBuiltinKeys;

@RunWith(MockitoJUnitRunner.class)
public class ProducerImplTest {
Expand All @@ -62,7 +61,7 @@ public class ProducerImplTest {
@Before
public void before() throws NoSuchFieldException, IllegalAccessException {
Properties config = new Properties();
config.setProperty(OMSBuiltinKeys.DRIVER_IMPL, "org.apache.eventmesh.connector.rocketmq.MessagingAccessPointImpl");
// config.setProperty(OMSBuiltinKeys.DRIVER_IMPL, "org.apache.eventmesh.connector.rocketmq.MessagingAccessPointImpl");
config.setProperty("access_points", "IP1:9876,IP2:9876");
producer = new ProducerImpl(config);

Expand Down Expand Up @@ -148,7 +147,6 @@ public void testSend_WithException() throws InterruptedException, RemotingExcept
Mockito.when(rocketmqProducer.send(any(Message.class))).thenThrow(exception);

try {
io.openmessaging.api.Message message = new io.openmessaging.api.Message("HELLO_TOPIC", "", new byte[]{'a'});
CloudEvent cloudEvent = CloudEventBuilder.v1()
.withId("id1")
.withSource(URI.create("https://github.com/cloudevents/*****"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ private static String generateRandomString(int length) {
return builder.toString();
}

public static CloudEvent generateCloudEventV1() {
public static CloudEvent generateCloudEventV1Async() {
Map<String, String> content = new HashMap<>();
content.put("content", "testAsyncMessage");

Expand All @@ -181,4 +181,22 @@ public static CloudEvent generateCloudEventV1() {
.build();
return event;
}

public static CloudEvent generateCloudEventV1SyncRR() {
Map<String, String> content = new HashMap<>();
content.put("content", "testSyncRR");

CloudEvent event = CloudEventBuilder.v1()
.withId(UUID.randomUUID().toString())
.withSubject(TOPIC_PRX_SyncSubscribeTest)
.withSource(URI.create("/"))
.withDataContentType("application/cloudevents+json")
.withType(EventMeshCommon.CLOUD_EVENTS_PROTOCOL_NAME)
.withData(JsonUtils.serialize(content).getBytes(StandardCharsets.UTF_8))
.withExtension("ttl", "30000")
.withExtension("msgtype", "persistent")
.withExtension("keys", generateRandomString(16))
.build();
return event;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public static void main(String[] agrs) throws Exception {
client.init();

for (int i = 0; i < 5; i++) {
CloudEvent event = EventMeshTestUtils.generateCloudEventV1();
CloudEvent event = EventMeshTestUtils.generateCloudEventV1Async();
logger.info("begin send async msg[{}]==================={}", i, event);
client.publish(event, EventMeshCommon.DEFAULT_TIME_OUT_MILLS);

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.eventmesh.tcp.demo.pub.cloudevents;

import org.apache.eventmesh.client.tcp.EventMeshTCPClient;
import org.apache.eventmesh.client.tcp.EventMeshTCPClientFactory;
import org.apache.eventmesh.client.tcp.common.EventMeshCommon;
import org.apache.eventmesh.client.tcp.conf.EventMeshTCPClientConfig;
import org.apache.eventmesh.common.protocol.tcp.EventMeshMessage;
import org.apache.eventmesh.common.protocol.tcp.Package;
import org.apache.eventmesh.common.protocol.tcp.UserAgent;
import org.apache.eventmesh.tcp.common.EventMeshTestUtils;

import io.cloudevents.CloudEvent;
import io.cloudevents.core.provider.EventFormatProvider;
import io.cloudevents.jackson.JsonFormat;
import lombok.extern.slf4j.Slf4j;

import java.nio.charset.StandardCharsets;

@Slf4j
public class SyncRequest {

private static EventMeshTCPClient<CloudEvent> client;

public static void main(String[] agrs) throws Exception {
UserAgent userAgent = EventMeshTestUtils.generateClient1();
EventMeshTCPClientConfig eventMeshTcpClientConfig = EventMeshTCPClientConfig.builder()
.host("127.0.0.1")
.port(10002)
.userAgent(userAgent)
.build();
try {
client = EventMeshTCPClientFactory.createEventMeshTCPClient(
eventMeshTcpClientConfig, CloudEvent.class);
client.init();

CloudEvent event = EventMeshTestUtils.generateCloudEventV1SyncRR();
log.info("begin send rr msg=================={}", event);
Package response = client.rr(event, EventMeshCommon.DEFAULT_TIME_OUT_MILLS);
CloudEvent replyEvent = EventFormatProvider.getInstance().resolveFormat(JsonFormat.CONTENT_TYPE)
.deserialize(response.getBody().toString().getBytes(StandardCharsets.UTF_8));
String content = new String(replyEvent.getData().toBytes(), StandardCharsets.UTF_8);
log.info("receive rr reply==================={}|{}", response, content);

} catch (Exception e) {
log.warn("SyncRequest failed", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,15 +31,18 @@
@Slf4j
public class SyncRequest {

private static EventMeshTCPClient<EventMeshMessage> client;

public static void main(String[] agrs) throws Exception {
UserAgent userAgent = EventMeshTestUtils.generateClient1();
EventMeshTCPClientConfig eventMeshTcpClientConfig = EventMeshTCPClientConfig.builder()
.host("127.0.0.1")
.port(10000)
.port(10002)
.userAgent(userAgent)
.build();
try (EventMeshTCPClient<EventMeshMessage> client = EventMeshTCPClientFactory.createEventMeshTCPClient(
eventMeshTcpClientConfig, EventMeshMessage.class)) {
try {
client = EventMeshTCPClientFactory.createEventMeshTCPClient(
eventMeshTcpClientConfig, EventMeshMessage.class);
client.init();

EventMeshMessage eventMeshMessage = EventMeshTestUtils.generateSyncRRMqMsg();
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.eventmesh.tcp.demo.sub.cloudevents;

import org.apache.eventmesh.client.tcp.EventMeshTCPClient;
import org.apache.eventmesh.client.tcp.EventMeshTCPClientFactory;
import org.apache.eventmesh.client.tcp.common.MessageUtils;
import org.apache.eventmesh.client.tcp.common.ReceiveMsgHook;
import org.apache.eventmesh.client.tcp.conf.EventMeshTCPClientConfig;
import org.apache.eventmesh.common.protocol.SubscriptionMode;
import org.apache.eventmesh.common.protocol.SubscriptionType;
import org.apache.eventmesh.common.protocol.tcp.Command;
import org.apache.eventmesh.common.protocol.tcp.EventMeshMessage;
import org.apache.eventmesh.common.protocol.tcp.Package;
import org.apache.eventmesh.common.protocol.tcp.UserAgent;
import org.apache.eventmesh.tcp.common.EventMeshTestUtils;

import io.cloudevents.CloudEvent;
import io.netty.channel.ChannelHandlerContext;
import lombok.extern.slf4j.Slf4j;

import java.nio.charset.StandardCharsets;

@Slf4j
public class SyncResponse implements ReceiveMsgHook<CloudEvent> {

public static SyncResponse handler = new SyncResponse();

private static EventMeshTCPClient<CloudEvent> client;

public static void main(String[] agrs) throws Exception {
UserAgent userAgent = EventMeshTestUtils.generateClient2();
EventMeshTCPClientConfig eventMeshTcpClientConfig = EventMeshTCPClientConfig.builder()
.host("127.0.0.1")
.port(10002)
.userAgent(userAgent)
.build();
try {
client = EventMeshTCPClientFactory
.createEventMeshTCPClient(eventMeshTcpClientConfig, CloudEvent.class);
client.init();

client.subscribe("TEST-TOPIC-TCP-SYNC", SubscriptionMode.CLUSTERING, SubscriptionType.SYNC);
// Synchronize RR messages
client.registerSubBusiHandler(handler);

client.listen();

} catch (Exception e) {
log.warn("SyncResponse failed", e);
}
}

@Override
public void handle(CloudEvent event, ChannelHandlerContext ctx) {
String content = new String(event.getData().toBytes(), StandardCharsets.UTF_8);
log.info("receive sync rr msg================{}|{}", event, content);
Package pkg = MessageUtils.buildPackage(event, Command.RESPONSE_TO_SERVER);
ctx.writeAndFlush(pkg);
}

}
Loading

0 comments on commit 46471ed

Please sign in to comment.