You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@rocketmq.apache.org by GitBox <gi...@apache.org> on 2022/02/28 03:31:08 UTC

[GitHub] [rocketmq-connect] zhaohai1299002788 opened a new pull request #1: add mns task

zhaohai1299002788 opened a new pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1


   add rocketmq-connect-mns


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] zhaohai1299002788 commented on a change in pull request #1: [ISSUE #11] Add mns source task

Posted by GitBox <gi...@apache.org>.
zhaohai1299002788 commented on a change in pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#discussion_r821371495



##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSSourceConnector.java
##########
@@ -0,0 +1,94 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.connector.ConnectorContext;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.source.SourceConnector;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.aliyun.rocketmq.connect.mns.source.constant.MNSConstant.*;
+
+public class MNSSourceConnector extends SourceConnector {
+
+    private String accessKeyId;
+
+    private String accessKeySecret;
+
+    private String accountEndpoint;
+
+    private String queueName;
+
+    private String accountId;
+
+    private String isBase64Decode;
+
+    private Integer batchSize;
+
+    @Override
+    public void pause() {
+
+    }
+
+    @Override
+    public void resume() {
+
+    }
+
+    @Override
+    public List<KeyValue> taskConfigs(int maxTasks) {
+        List<KeyValue> taskConfigList = new ArrayList<>(11);
+        KeyValue keyValue = new DefaultKeyValue();
+        keyValue.put(ACCESS_KEY_ID, accessKeyId);
+        keyValue.put(ACCESS_KEY_SECRET, accessKeySecret);
+        keyValue.put(ACCOUNT_ENDPOINT, accountEndpoint);
+        keyValue.put(QUEUE_NAME, queueName);
+        keyValue.put(ACCOUNT_ID, accountId);
+        if (batchSize == null) {
+            keyValue.put(BATCH_SIZE, 8);
+        }
+        keyValue.put(IS_BASE64_DECODE, isBase64Decode);
+        taskConfigList.add(keyValue);
+        return taskConfigList;
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return MNSSourceTask.class;
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(ACCESS_KEY_ID))
+                || StringUtils.isBlank(config.getString(ACCESS_KEY_SECRET))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ENDPOINT))
+                || StringUtils.isBlank(config.getString(QUEUE_NAME))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ID))) {
+            throw new RuntimeException("mns required parameter is null !");
+        }

Review comment:
       Queue name check added




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] zhaohai1299002788 commented on a change in pull request #1: [ISSUE #11] Add mns source task

Posted by GitBox <gi...@apache.org>.
zhaohai1299002788 commented on a change in pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#discussion_r821371737



##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSSourceConnector.java
##########
@@ -0,0 +1,94 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.connector.ConnectorContext;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.source.SourceConnector;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.aliyun.rocketmq.connect.mns.source.constant.MNSConstant.*;
+
+public class MNSSourceConnector extends SourceConnector {
+
+    private String accessKeyId;
+
+    private String accessKeySecret;
+
+    private String accountEndpoint;
+
+    private String queueName;
+
+    private String accountId;
+
+    private String isBase64Decode;
+
+    private Integer batchSize;
+
+    @Override
+    public void pause() {
+
+    }
+
+    @Override
+    public void resume() {
+
+    }
+
+    @Override
+    public List<KeyValue> taskConfigs(int maxTasks) {
+        List<KeyValue> taskConfigList = new ArrayList<>(11);
+        KeyValue keyValue = new DefaultKeyValue();
+        keyValue.put(ACCESS_KEY_ID, accessKeyId);
+        keyValue.put(ACCESS_KEY_SECRET, accessKeySecret);
+        keyValue.put(ACCOUNT_ENDPOINT, accountEndpoint);
+        keyValue.put(QUEUE_NAME, queueName);
+        keyValue.put(ACCOUNT_ID, accountId);
+        if (batchSize == null) {
+            keyValue.put(BATCH_SIZE, 8);
+        }
+        keyValue.put(IS_BASE64_DECODE, isBase64Decode);
+        taskConfigList.add(keyValue);
+        return taskConfigList;
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return MNSSourceTask.class;
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(ACCESS_KEY_ID))
+                || StringUtils.isBlank(config.getString(ACCESS_KEY_SECRET))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ENDPOINT))
+                || StringUtils.isBlank(config.getString(QUEUE_NAME))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ID))) {
+            throw new RuntimeException("mns required parameter is null !");
+        }
+    }
+
+    @Override
+    public void init(KeyValue config) {
+        accessKeyId = config.getString(ACCESS_KEY_ID);
+        accessKeySecret = config.getString(ACCESS_KEY_SECRET);
+        accountEndpoint = config.getString(ACCOUNT_ENDPOINT);
+        queueName = config.getString(QUEUE_NAME);
+        batchSize = config.getInt(BATCH_SIZE, 8);
+        accountId = config.getString(ACCOUNT_ID);

Review comment:
       delete duplicate code




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] zhaohai1299002788 commented on a change in pull request #1: [ISSUE #11] Add mns source task

Posted by GitBox <gi...@apache.org>.
zhaohai1299002788 commented on a change in pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#discussion_r821417006



##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSRecordConverImpl.java
##########
@@ -0,0 +1,69 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import com.aliyun.mns.model.Message;
+import com.aliyun.rocketmq.connect.mns.source.enums.CloudEventsEnum;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonPrimitive;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public class MNSRecordConverImpl extends AbstractMNSRecordConvert {
+
+
+    @Override
+    public void fillCloudEventsKey(ConnectRecord connectRecord, String regionId, String accountId, String queueName, Message popMsg, boolean isBase64Secode) {
+        String messageBodyValue = "";
+        if (isBase64Secode) {
+            messageBodyValue = new String(popMsg.getMessageBodyAsBytes(), StandardCharsets.UTF_8);
+        } else {
+            messageBodyValue = new String(popMsg.getMessageBodyAsRawBytes(), StandardCharsets.UTF_8);
+        }
+        JsonElement messageBody = null;
+        try {
+            messageBody = parseToJsonElement(messageBodyValue);
+        } catch (Exception e) {
+            messageBody = new JsonPrimitive(messageBodyValue);
+        }
+        String eventId;
+        if (StringUtils.isBlank(popMsg.getMessageId())) {
+            eventId = popMsg.getMessageId();
+        } else {
+            eventId = UUID.randomUUID().toString();
+        }
+        connectRecord.addExtension(CloudEventsEnum.CE_ID.getCode(), eventId);

Review comment:
       delete cloud events




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] 2011shenlin commented on a change in pull request #1: [ISSUE #11] Add mns source task

Posted by GitBox <gi...@apache.org>.
2011shenlin commented on a change in pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#discussion_r821299948



##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSSourceConnector.java
##########
@@ -0,0 +1,94 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.connector.ConnectorContext;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.source.SourceConnector;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.aliyun.rocketmq.connect.mns.source.constant.MNSConstant.*;
+
+public class MNSSourceConnector extends SourceConnector {
+
+    private String accessKeyId;
+
+    private String accessKeySecret;
+
+    private String accountEndpoint;
+
+    private String queueName;
+
+    private String accountId;
+
+    private String isBase64Decode;
+
+    private Integer batchSize;
+
+    @Override
+    public void pause() {
+
+    }
+
+    @Override
+    public void resume() {
+
+    }
+
+    @Override
+    public List<KeyValue> taskConfigs(int maxTasks) {
+        List<KeyValue> taskConfigList = new ArrayList<>(11);
+        KeyValue keyValue = new DefaultKeyValue();
+        keyValue.put(ACCESS_KEY_ID, accessKeyId);
+        keyValue.put(ACCESS_KEY_SECRET, accessKeySecret);
+        keyValue.put(ACCOUNT_ENDPOINT, accountEndpoint);
+        keyValue.put(QUEUE_NAME, queueName);
+        keyValue.put(ACCOUNT_ID, accountId);
+        if (batchSize == null) {
+            keyValue.put(BATCH_SIZE, 8);
+        }
+        keyValue.put(IS_BASE64_DECODE, isBase64Decode);
+        taskConfigList.add(keyValue);
+        return taskConfigList;
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return MNSSourceTask.class;
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(ACCESS_KEY_ID))
+                || StringUtils.isBlank(config.getString(ACCESS_KEY_SECRET))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ENDPOINT))
+                || StringUtils.isBlank(config.getString(QUEUE_NAME))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ID))) {
+            throw new RuntimeException("mns required parameter is null !");
+        }

Review comment:
       check the queue name existed.

##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSSourceConnector.java
##########
@@ -0,0 +1,94 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.connector.ConnectorContext;
+import io.openmessaging.connector.api.component.task.Task;
+import io.openmessaging.connector.api.component.task.source.SourceConnector;
+import io.openmessaging.internal.DefaultKeyValue;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static com.aliyun.rocketmq.connect.mns.source.constant.MNSConstant.*;
+
+public class MNSSourceConnector extends SourceConnector {
+
+    private String accessKeyId;
+
+    private String accessKeySecret;
+
+    private String accountEndpoint;
+
+    private String queueName;
+
+    private String accountId;
+
+    private String isBase64Decode;
+
+    private Integer batchSize;
+
+    @Override
+    public void pause() {
+
+    }
+
+    @Override
+    public void resume() {
+
+    }
+
+    @Override
+    public List<KeyValue> taskConfigs(int maxTasks) {
+        List<KeyValue> taskConfigList = new ArrayList<>(11);
+        KeyValue keyValue = new DefaultKeyValue();
+        keyValue.put(ACCESS_KEY_ID, accessKeyId);
+        keyValue.put(ACCESS_KEY_SECRET, accessKeySecret);
+        keyValue.put(ACCOUNT_ENDPOINT, accountEndpoint);
+        keyValue.put(QUEUE_NAME, queueName);
+        keyValue.put(ACCOUNT_ID, accountId);
+        if (batchSize == null) {
+            keyValue.put(BATCH_SIZE, 8);
+        }
+        keyValue.put(IS_BASE64_DECODE, isBase64Decode);
+        taskConfigList.add(keyValue);
+        return taskConfigList;
+    }
+
+    @Override
+    public Class<? extends Task> taskClass() {
+        return MNSSourceTask.class;
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(ACCESS_KEY_ID))
+                || StringUtils.isBlank(config.getString(ACCESS_KEY_SECRET))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ENDPOINT))
+                || StringUtils.isBlank(config.getString(QUEUE_NAME))
+                || StringUtils.isBlank(config.getString(ACCOUNT_ID))) {
+            throw new RuntimeException("mns required parameter is null !");
+        }
+    }
+
+    @Override
+    public void init(KeyValue config) {
+        accessKeyId = config.getString(ACCESS_KEY_ID);
+        accessKeySecret = config.getString(ACCESS_KEY_SECRET);
+        accountEndpoint = config.getString(ACCOUNT_ENDPOINT);
+        queueName = config.getString(QUEUE_NAME);
+        batchSize = config.getInt(BATCH_SIZE, 8);
+        accountId = config.getString(ACCOUNT_ID);

Review comment:
       If the configuration items of SinkConnector and SinkTask are the same, could return the SinkConnector config directly,Avoid duplicate code between SinkConnector and SinkTask about configuration item extraction

##########
File path: connectors/aliyun/rocketmq-connect-mns/src/main/java/com/aliyun/rocketmq/connect/mns/source/MNSRecordConverImpl.java
##########
@@ -0,0 +1,69 @@
+package com.aliyun.rocketmq.connect.mns.source;
+
+import com.aliyun.mns.model.Message;
+import com.aliyun.rocketmq.connect.mns.source.enums.CloudEventsEnum;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParser;
+import com.google.gson.JsonPrimitive;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.SchemaBuilder;
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+public class MNSRecordConverImpl extends AbstractMNSRecordConvert {
+
+
+    @Override
+    public void fillCloudEventsKey(ConnectRecord connectRecord, String regionId, String accountId, String queueName, Message popMsg, boolean isBase64Secode) {
+        String messageBodyValue = "";
+        if (isBase64Secode) {
+            messageBodyValue = new String(popMsg.getMessageBodyAsBytes(), StandardCharsets.UTF_8);
+        } else {
+            messageBodyValue = new String(popMsg.getMessageBodyAsRawBytes(), StandardCharsets.UTF_8);
+        }
+        JsonElement messageBody = null;
+        try {
+            messageBody = parseToJsonElement(messageBodyValue);
+        } catch (Exception e) {
+            messageBody = new JsonPrimitive(messageBodyValue);
+        }
+        String eventId;
+        if (StringUtils.isBlank(popMsg.getMessageId())) {
+            eventId = popMsg.getMessageId();
+        } else {
+            eventId = UUID.randomUUID().toString();
+        }
+        connectRecord.addExtension(CloudEventsEnum.CE_ID.getCode(), eventId);

Review comment:
       remove this, and move to the cloudevent transform plugin.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] zhaohai1299002788 commented on pull request #1: add mns task

Posted by GitBox <gi...@apache.org>.
zhaohai1299002788 commented on pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#issuecomment-1053848816


   <img width="1095" alt="image" src="https://user-images.githubusercontent.com/33314633/155919784-8c02c42d-ed8b-434d-a4c4-1355bec8dae5.png">
   这个问题是重写commit的时候,参数List集合无法带范型。


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq-connect] zhaohai1299002788 removed a comment on pull request #1: add mns task

Posted by GitBox <gi...@apache.org>.
zhaohai1299002788 removed a comment on pull request #1:
URL: https://github.com/apache/rocketmq-connect/pull/1#issuecomment-1053848816


   <img width="1095" alt="image" src="https://user-images.githubusercontent.com/33314633/155919784-8c02c42d-ed8b-434d-a4c4-1355bec8dae5.png">
   这个问题是重写commit的时候,参数List集合无法带范型。


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org