You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by GitBox <gi...@apache.org> on 2022/11/17 08:32:36 UTC

[GitHub] [rocketmq-connect] fireroundgithub opened a new pull request, #382: [#366] Support for sftp connector(sinlk & source)

fireroundgithub opened a new pull request, #382:
URL: https://github.com/apache/rocketmq-connect/pull/382

   ## What is the purpose of the change
   
   close #366 
   
   ## Brief changelog
   
   Basic support for sftp connector.
   
   ## Verifying this change
   
   XXXX
   
   Follow this checklist to help us incorporate your contribution quickly and easily. Notice, `it would be helpful if you could finish the following 5 checklist(the last one is not necessary)before request the community to review your PR`.
   
   - [x] Make sure there is a [Github issue](https://github.com/apache/rocketmq-connect/issues) filed for the change (usually before you start working on it). Trivial changes like typos do not require a Github issue. Your pull request should address just this issue, without pulling in other changes - one PR resolves one issue. 
   - [x] Format the pull request title like `[ISSUE #123] Fix UnknownException when host config not exist`. Each commit in the pull request should have a meaningful subject line and body.
   - [x] Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
   - [x] Write necessary unit-test(over 80% coverage) to verify your logic correction, more mock a little better when cross module dependency exist. If the new feature or significant change is committed, please remember to add integration-test in [test module](https://github.com/apache/rocketmq/tree/master/test).
   - [x] Run `mvn -B clean apache-rat:check findbugs:findbugs checkstyle:checkstyle` to make sure basic checks pass. Run `mvn clean install -DskipITs` to make sure unit-test pass. Run `mvn clean test-compile failsafe:integration-test`  to make sure integration-test pass.
   - [ ] If this contribution is large, please file an [Apache Individual Contributor License Agreement](http://www.apache.org/licenses/#clas).
   


-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] fireroundgithub commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
fireroundgithub commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1030098416


##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSourceTask.java:
##########
@@ -0,0 +1,120 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.file.FileSystemException;
+import java.util.*;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSourceTask extends SourceTask {
+
+    private final Logger log = LoggerFactory.getLogger(SftpConstant.LOGGER_NAME);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    private static final int MAX_NUMBER_SEND_CONNECT_RECORD_EACH_TIME = 2000;
+
+    @Override
+    public void init(SourceTaskContext sourceTaskContext) {
+        super.init(sourceTaskContext);
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+    }

Review Comment:
   Are you think it will be better to close sftpClient when stopping a task?



-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] fireroundgithub commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
fireroundgithub commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1030089318


##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSinkTask.java:
##########
@@ -0,0 +1,67 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.sink.SinkTask;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSinkTask extends SinkTask {
+    private static final Logger log = LoggerFactory.getLogger(SftpSinkTask.class);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    @Override
+    public void put(List<ConnectRecord> sinkRecords) throws ConnectException {
+        try {
+            for (ConnectRecord connectRecord : sinkRecords) {
+                String str = (String) connectRecord.getData();
+                str = str + System.lineSeparator();
+                sftpClient.append(new ByteArrayInputStream(str.getBytes()), filename);
+            }
+        } catch (IOException e) {
+            log.error("sink task ioexception", e);
+        } finally {
+            sftpClient.close();
+        }
+        sinkRecords.forEach(System.out::println);
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(SFTP_HOST_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PORT_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_USERNAME_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PASSWORD_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PATH_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_FILENAME_KEY))) {
+            throw new RuntimeException("missing required config");
+        }
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+

Review Comment:
   It will be close after poll.



##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSourceTask.java:
##########
@@ -0,0 +1,120 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.file.FileSystemException;
+import java.util.*;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSourceTask extends SourceTask {
+
+    private final Logger log = LoggerFactory.getLogger(SftpConstant.LOGGER_NAME);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    private static final int MAX_NUMBER_SEND_CONNECT_RECORD_EACH_TIME = 2000;
+
+    @Override
+    public void init(SourceTaskContext sourceTaskContext) {
+        super.init(sourceTaskContext);
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+    }

Review Comment:
   It will be close after poll.



-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] odbozhou merged pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
odbozhou merged PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382


-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] fireroundgithub commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
fireroundgithub commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1030097689


##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpClient.java:
##########
@@ -0,0 +1,74 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import com.jcraft.jsch.*;

Review Comment:
   Thanks. I will import code style configuration file to my IDEA, then reformat it.



-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] odbozhou commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
odbozhou commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1028904915


##########
connectors/rocketmq-connect-sftp/README.md:
##########
@@ -0,0 +1,96 @@
+# rocketmq-connect-sftp
+
+Plugin for Rocketmq Connect. Tansfer file based on SFTP.
+
+# How to use
+* start rocketmq nameserver
+```shell
+nohup ./bin/mqnamesrv &
+```
+* start rocketmq broker
+```shell
+nohup ./bin/mqbroker -n localhost:9876 &
+```
+* build plugin
+```shell
+cd connectors/rocketmq-connect-sftp
+mvn clean install -Dmaven.test.skip=true
+```
+* create config file path/to/connect-standalone.conf same as distribution/conf/connect-standalone.conf
+* modify pluginPaths=path/to/rocketmq-connect-sftp-0.0.1-SNAPSHOT-jar-with-dependencies
+* start org.apache.rocketmq.connect.runtime.StandaloneConnectStartup
+```shell
+cd rocketmq-connect-runtime
+mvn clean install -Dmaven.test.skip=true
+```
+* start source connector
+```http request
+POST /connectors/SftpSourceConnector HTTP/1.1
+Host: localhost:8082
+Content-Type: application/json
+
+{
+  "connector.class": "org.apache.rocketmq.connect.http.sink.SftpSourceConnector",
+  "host": "127.0.0.1",
+  "port": 22,
+  "filename": "source.txt",
+  "username": "wencheng",
+  "password": "",
+  "path": "Documents",
+  "connect.topicname": "sftpTopic"
+}
+```
+
+`warning:` make sure exist a file named "source.txt" on the sftp server.
+
+* start sink connector
+```http request
+POST /connectors/SftpSinkConnector HTTP/1.1
+Host: localhost:8082
+Content-Type: application/json
+
+{
+  "connector.class": "org.apache.rocketmq.connect.http.sink.SftpSinkConnector",
+  "host": "127.0.0.1",
+  "port": 22,
+  "filename": "sink.txt",
+  "username": "wencheng",
+  "password": "",
+  "connect.topicname": "sftpTopic"
+}
+```
+
+## What we expected to see
+The file named sink.txt will be created, and the content of the "source.txt" will appears in this file.
+
+
+## Appendix: Connector Configuration
+
+### sftp-source-connector configuration
+
+| KEY      | TYPE   | REQUIRED | DESCRIPTION                                            | EXAMPLE               |
+|----------|--------|----------|--------------------------------------------------------|-----------------------|
+| host     | String | Y        | SFTP host                                              | localhost             |
+| port     | int    | Y        | SFTP port                                              | 22                    |
+| username | String | Y        | SFTP username                                          | wencheng              |
+| password | String | Y        | SFTP password                                          |                       |
+| path     | String | Y        | SFTP path                                              | /Documents            |
+| filename | String | Y        | The name of the file which will be transferred         | source.txt            |
+| topic    | String | Y        | The Message Queue topic which the data will be send to |                       |

Review Comment:
   connect.topicname ?



##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpClient.java:
##########
@@ -0,0 +1,74 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import com.jcraft.jsch.*;

Review Comment:
   Please import the specific reference



##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSourceTask.java:
##########
@@ -0,0 +1,120 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.file.FileSystemException;
+import java.util.*;

Review Comment:
   Please import the specific reference



##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSourceTask.java:
##########
@@ -0,0 +1,120 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.file.FileSystemException;
+import java.util.*;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSourceTask extends SourceTask {
+
+    private final Logger log = LoggerFactory.getLogger(SftpConstant.LOGGER_NAME);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    private static final int MAX_NUMBER_SEND_CONNECT_RECORD_EACH_TIME = 2000;
+
+    @Override
+    public void init(SourceTaskContext sourceTaskContext) {
+        super.init(sourceTaskContext);
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+    }

Review Comment:
   Do you need to close sftpClient when stopping a task?



##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSinkTask.java:
##########
@@ -0,0 +1,67 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.sink.SinkTask;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSinkTask extends SinkTask {
+    private static final Logger log = LoggerFactory.getLogger(SftpSinkTask.class);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    @Override
+    public void put(List<ConnectRecord> sinkRecords) throws ConnectException {
+        try {
+            for (ConnectRecord connectRecord : sinkRecords) {
+                String str = (String) connectRecord.getData();
+                str = str + System.lineSeparator();
+                sftpClient.append(new ByteArrayInputStream(str.getBytes()), filename);
+            }
+        } catch (IOException e) {
+            log.error("sink task ioexception", e);
+        } finally {
+            sftpClient.close();
+        }
+        sinkRecords.forEach(System.out::println);
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(SFTP_HOST_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PORT_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_USERNAME_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PASSWORD_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PATH_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_FILENAME_KEY))) {
+            throw new RuntimeException("missing required config");
+        }
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+

Review Comment:
   Do you need to close sftpClient when stopping a task?



-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] fireroundgithub commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
fireroundgithub commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1030089318


##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSinkTask.java:
##########
@@ -0,0 +1,67 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.sink.SinkTask;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.errors.ConnectException;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSinkTask extends SinkTask {
+    private static final Logger log = LoggerFactory.getLogger(SftpSinkTask.class);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    @Override
+    public void put(List<ConnectRecord> sinkRecords) throws ConnectException {
+        try {
+            for (ConnectRecord connectRecord : sinkRecords) {
+                String str = (String) connectRecord.getData();
+                str = str + System.lineSeparator();
+                sftpClient.append(new ByteArrayInputStream(str.getBytes()), filename);
+            }
+        } catch (IOException e) {
+            log.error("sink task ioexception", e);
+        } finally {
+            sftpClient.close();
+        }
+        sinkRecords.forEach(System.out::println);
+    }
+
+    @Override
+    public void validate(KeyValue config) {
+        if (StringUtils.isBlank(config.getString(SFTP_HOST_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PORT_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_USERNAME_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PASSWORD_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_PATH_KEY))
+                || StringUtils.isBlank(config.getString(SFTP_FILENAME_KEY))) {
+            throw new RuntimeException("missing required config");
+        }
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+

Review Comment:
   It will be close after put.



-- 
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: commits-unsubscribe@rocketmq.apache.org

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


[GitHub] [rocketmq-connect] odbozhou commented on a diff in pull request #382: [ISSUE #366] Support for sftp connector(sinlk & source)

Posted by GitBox <gi...@apache.org>.
odbozhou commented on code in PR #382:
URL: https://github.com/apache/rocketmq-connect/pull/382#discussion_r1031173503


##########
connectors/rocketmq-connect-sftp/src/main/java/org/apache/rocketmq/connect/http/sink/SftpSourceTask.java:
##########
@@ -0,0 +1,120 @@
+package org.apache.rocketmq.connect.http.sink;
+
+import io.openmessaging.KeyValue;
+import io.openmessaging.connector.api.component.task.source.SourceTask;
+import io.openmessaging.connector.api.component.task.source.SourceTaskContext;
+import io.openmessaging.connector.api.data.ConnectRecord;
+import io.openmessaging.connector.api.data.RecordOffset;
+import io.openmessaging.connector.api.data.RecordPartition;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.nio.file.FileSystemException;
+import java.util.*;
+
+import static org.apache.rocketmq.connect.http.sink.SftpConstant.*;
+
+public class SftpSourceTask extends SourceTask {
+
+    private final Logger log = LoggerFactory.getLogger(SftpConstant.LOGGER_NAME);
+
+    private SftpClient sftpClient;
+
+    private String filename;
+
+    private static final int MAX_NUMBER_SEND_CONNECT_RECORD_EACH_TIME = 2000;
+
+    @Override
+    public void init(SourceTaskContext sourceTaskContext) {
+        super.init(sourceTaskContext);
+    }
+
+    @Override
+    public void start(KeyValue config) {
+        String host = config.getString(SFTP_HOST_KEY);
+        int port = config.getInt(SFTP_PORT_KEY);
+        String username = config.getString(SFTP_USERNAME_KEY);
+        String password = config.getString(SFTP_PASSWORD_KEY);
+        String path = config.getString(SFTP_PATH_KEY);
+        filename = config.getString(SFTP_FILENAME_KEY);
+        sftpClient = new SftpClient(host, port, username, password, path);
+    }
+
+    @Override
+    public void stop() {
+    }

Review Comment:
   Create an sftp session every time you poll or put data, and close the connection after the batch of messages are processed. Will it affect the processing efficiency? Is it possible to create a connection when the task is started, and close the connection when the task is stopped? Considering the sftp link disconnection, it is possible to re-create the connection when it is detected that the connection is unavailable. Does this take performance and availability into account? These are just some of my suggestions, if possible, consider implementing them in the next version.



-- 
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: commits-unsubscribe@rocketmq.apache.org

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