You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by GitBox <gi...@apache.org> on 2022/04/08 13:58:08 UTC

[GitHub] [incubator-seatunnel] tmljob opened a new pull request, #1678: [Feature#1270][Connectors] add plugin: seatunnel-connector-http

tmljob opened a new pull request, #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678

   <!--
   
   Thank you for contributing to SeaTunnel! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [GITHUB issue](https://github.com/apache/incubator-seatunnel/issues).
   
     - Name the pull request in the form "[Feature] [component] Title of the pull request", where *Feature* can be replaced by `Hotfix`, `Bug`, etc.
   
     - Minor fixes should be named following this pattern: `[hotfix] [docs] Fix typo in README.md doc`.
   
   -->
   
   ## Purpose of this pull request
   
   <!-- Describe the purpose of this pull request. For example: This pull request adds checkstyle plugin.-->
   
   ## Check list
   
   * [ ] Code changed are covered with tests, or it does not need tests for reason:
   * [ ] If any new Jar binary package adding in your PR, please add License Notice according
     [New License Guide](https://github.com/apache/incubator-seatunnel/blob/dev/docs/en/development/new-license.md)
   * [ ] If necessary, please update the documentation to describe the new feature. https://github.com/apache/incubator-seatunnel/tree/dev/docs
   


-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847298399


##########
seatunnel-connectors/seatunnel-connectors-spark/pom.xml:
##########
@@ -18,40 +18,41 @@
 
 -->
 <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>
-        <groupId>org.apache.seatunnel</groupId>
-        <artifactId>seatunnel-connectors</artifactId>
-        <version>${revision}</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

Review Comment:
   It seems our code style doesn't cover the of pom, but you don't need to do this change, it's better to revert this change.



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] tmljob commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
tmljob commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847272396


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;
+
+import org.apache.seatunnel.common.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.spark.batch.SparkBatchSource;
+import org.apache.seatunnel.spark.source.constant.Settings;
+import org.apache.seatunnel.spark.source.util.HttpClientResult;
+import org.apache.seatunnel.spark.source.util.HttpClientUtils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Http extends SparkBatchSource {
+
+    private static final String GET = "GET";
+    private static final String POST = "POST";
+    private static final int INITIAL_CAPACITY = 16;
+
+    private static Logger LOG = LoggerFactory.getLogger(Http.class);
+
+    @Override
+    public CheckResult checkConfig() {
+        return CheckConfigUtil.checkAllExists(config, "url");
+    }
+
+    @Override
+    public void prepare(SparkEnvironment prepareEnv) {
+    }
+
+    @Override
+    public Dataset<Row> getData(SparkEnvironment env) {
+        SparkSession spark = env.getSparkSession();
+        String url = config.getString(Settings.SOURCE_HTTP_URL);
+        String method = config.hasPath(Settings.SOURCE_HTTP_METHOD) ?
+            config.getString(Settings.SOURCE_HTTP_METHOD) : GET;
+        String header = config.hasPath(Settings.SOURCE_HTTP_HEADER) ?
+            config.getString(Settings.SOURCE_HTTP_HEADER) : "";
+        String requestParams = config.hasPath(Settings.SOURCE_HTTP_REQUEST_PARAMS) ?
+            config.getString(Settings.SOURCE_HTTP_REQUEST_PARAMS) : "";
+        String syncPath = config.hasPath(Settings.SOURCE_HTTP_SYNC_PATH) ?
+            config.getString(Settings.SOURCE_HTTP_SYNC_PATH) : "";
+
+        JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+        Map requestMap = jsonToMap(requestParams);
+        String syncValues = getSyncValues(jsc, syncPath);
+        LOG.info("sync values->{}", syncValues);
+        Map syncMap = jsonToMap(syncValues);
+        if (!syncMap.isEmpty()) {
+            requestMap.putAll(syncMap);
+        }
+
+        HttpClientResult response = new HttpClientResult();
+        try {
+            Map headerMap = jsonToMap(header);
+            if (POST.equals(method)) {
+                response = HttpClientUtils.doPost(url, headerMap, requestMap);
+            } else {
+                response = HttpClientUtils.doGet(url, headerMap, requestMap);
+            }
+        } catch (Exception e) {
+            LOG.error("http call error!", e);
+        }
+
+        LOG.info("http respond code->{}", response.getCode());
+
+        List<String> array = new ArrayList<>();
+        array.add(response.getContent());
+        JavaRDD<String> javaRDD = jsc.parallelize(array);
+        DataFrameReader reader = spark.read().format("json");
+        return reader.json(javaRDD);
+    }
+
+    private String getSyncValues(JavaSparkContext jsc, String syncPath) {
+        if (null == syncPath || syncPath.isEmpty()) {
+            return "";
+        }
+        Configuration hadoopConf = jsc.hadoopConfiguration();
+        List<String> values = new ArrayList<>();
+        try {
+            FileSystem fs = FileSystem.get(hadoopConf);
+            Path path = new Path(syncPath);
+            boolean exists = fs.exists(path);
+            if (exists) {
+                JavaRDD<String> checkPoint = jsc.textFile(syncPath);
+                values = checkPoint.collect();
+
+            }
+        } catch (IOException e) {
+            LOG.error("getSyncValues error, syncPath is {}", syncPath, e);
+        }
+        return values.isEmpty() ? "" : values.iterator().next();
+    }
+
+    private Map jsonToMap(String content) {
+        Map map = new HashMap<>(INITIAL_CAPACITY);
+        if (null == content || content.isEmpty()) {
+            return map;
+        }
+
+        try {
+            return new ObjectMapper().readValue(content, HashMap.class);
+        } catch (IOException e) {
+            LOG.error("{} json to map error!", content, e);

Review Comment:
   
   Annotation has been 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: commits-unsubscribe@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r846919934


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;
+
+import org.apache.seatunnel.common.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.spark.batch.SparkBatchSource;
+import org.apache.seatunnel.spark.source.constant.Settings;
+import org.apache.seatunnel.spark.source.util.HttpClientResult;
+import org.apache.seatunnel.spark.source.util.HttpClientUtils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Http extends SparkBatchSource {
+
+    private static final String GET = "GET";
+    private static final String POST = "POST";
+    private static final int INITIAL_CAPACITY = 16;
+
+    private static Logger LOG = LoggerFactory.getLogger(Http.class);
+
+    @Override
+    public CheckResult checkConfig() {
+        return CheckConfigUtil.checkAllExists(config, "url");
+    }
+
+    @Override
+    public void prepare(SparkEnvironment prepareEnv) {
+    }
+
+    @Override
+    public Dataset<Row> getData(SparkEnvironment env) {
+        SparkSession spark = env.getSparkSession();
+        String url = config.getString(Settings.SOURCE_HTTP_URL);
+        String method = config.hasPath(Settings.SOURCE_HTTP_METHOD) ?
+            config.getString(Settings.SOURCE_HTTP_METHOD) : GET;
+        String header = config.hasPath(Settings.SOURCE_HTTP_HEADER) ?
+            config.getString(Settings.SOURCE_HTTP_HEADER) : "";
+        String requestParams = config.hasPath(Settings.SOURCE_HTTP_REQUEST_PARAMS) ?
+            config.getString(Settings.SOURCE_HTTP_REQUEST_PARAMS) : "";
+        String syncPath = config.hasPath(Settings.SOURCE_HTTP_SYNC_PATH) ?
+            config.getString(Settings.SOURCE_HTTP_SYNC_PATH) : "";
+
+        JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+        Map requestMap = jsonToMap(requestParams);
+        String syncValues = getSyncValues(jsc, syncPath);
+        LOG.info("sync values->{}", syncValues);
+        Map syncMap = jsonToMap(syncValues);
+        if (!syncMap.isEmpty()) {
+            requestMap.putAll(syncMap);
+        }
+
+        HttpClientResult response = new HttpClientResult();
+        try {
+            Map headerMap = jsonToMap(header);
+            if (POST.equals(method)) {
+                response = HttpClientUtils.doPost(url, headerMap, requestMap);
+            } else {
+                response = HttpClientUtils.doGet(url, headerMap, requestMap);
+            }
+        } catch (Exception e) {
+            LOG.error("http call error!", e);
+        }
+
+        LOG.info("http respond code->{}", response.getCode());
+
+        List<String> array = new ArrayList<>();
+        array.add(response.getContent());
+        JavaRDD<String> javaRDD = jsc.parallelize(array);
+        DataFrameReader reader = spark.read().format("json");
+        return reader.json(javaRDD);
+    }
+
+    private String getSyncValues(JavaSparkContext jsc, String syncPath) {
+        if (null == syncPath || syncPath.isEmpty()) {
+            return "";
+        }
+        Configuration hadoopConf = jsc.hadoopConfiguration();
+        List<String> values = new ArrayList<>();
+        try {
+            FileSystem fs = FileSystem.get(hadoopConf);
+            Path path = new Path(syncPath);
+            boolean exists = fs.exists(path);
+            if (exists) {
+                JavaRDD<String> checkPoint = jsc.textFile(syncPath);
+                values = checkPoint.collect();
+
+            }
+        } catch (IOException e) {
+            LOG.error("getSyncValues error, syncPath is {}", syncPath, e);
+        }
+        return values.isEmpty() ? "" : values.iterator().next();
+    }
+
+    private Map jsonToMap(String content) {
+        Map map = new HashMap<>(INITIAL_CAPACITY);
+        if (null == content || content.isEmpty()) {
+            return map;
+        }
+
+        try {
+            return new ObjectMapper().readValue(content, HashMap.class);
+        } catch (IOException e) {
+            LOG.error("{} json to map error!", content, e);

Review Comment:
   Why catch this exception, make this case to be normal? I think in most case, if you think this exception just need to be resolved in this method, you need to add a comment why this exception needn't to throw.



##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;

Review Comment:
   It's better to change the package name to `org.apache.seatunnel.spark.http.source`.



##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;
+
+import org.apache.seatunnel.common.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.spark.batch.SparkBatchSource;
+import org.apache.seatunnel.spark.source.constant.Settings;
+import org.apache.seatunnel.spark.source.util.HttpClientResult;
+import org.apache.seatunnel.spark.source.util.HttpClientUtils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Http extends SparkBatchSource {
+
+    private static final String GET = "GET";
+    private static final String POST = "POST";
+    private static final int INITIAL_CAPACITY = 16;
+
+    private static Logger LOG = LoggerFactory.getLogger(Http.class);
+
+    @Override
+    public CheckResult checkConfig() {
+        return CheckConfigUtil.checkAllExists(config, "url");
+    }
+
+    @Override
+    public void prepare(SparkEnvironment prepareEnv) {
+    }
+
+    @Override
+    public Dataset<Row> getData(SparkEnvironment env) {
+        SparkSession spark = env.getSparkSession();
+        String url = config.getString(Settings.SOURCE_HTTP_URL);
+        String method = config.hasPath(Settings.SOURCE_HTTP_METHOD) ?
+            config.getString(Settings.SOURCE_HTTP_METHOD) : GET;
+        String header = config.hasPath(Settings.SOURCE_HTTP_HEADER) ?
+            config.getString(Settings.SOURCE_HTTP_HEADER) : "";
+        String requestParams = config.hasPath(Settings.SOURCE_HTTP_REQUEST_PARAMS) ?
+            config.getString(Settings.SOURCE_HTTP_REQUEST_PARAMS) : "";
+        String syncPath = config.hasPath(Settings.SOURCE_HTTP_SYNC_PATH) ?
+            config.getString(Settings.SOURCE_HTTP_SYNC_PATH) : "";

Review Comment:
   It's better to use `TypesafeConfigUtils#getConfig(final Config config, final String configKey, final T defaultValue)` , this is more readable.
   e.g.
   ```java
   TypesafeConfigUtils.getConfig(config, Settings.SOURCE_HTTP_METHOD, GET);
   ```



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847169777


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;
+
+import org.apache.seatunnel.common.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.spark.batch.SparkBatchSource;
+import org.apache.seatunnel.spark.source.constant.Settings;
+import org.apache.seatunnel.spark.source.util.HttpClientResult;
+import org.apache.seatunnel.spark.source.util.HttpClientUtils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Http extends SparkBatchSource {
+
+    private static final String GET = "GET";
+    private static final String POST = "POST";
+    private static final int INITIAL_CAPACITY = 16;
+
+    private static Logger LOG = LoggerFactory.getLogger(Http.class);
+
+    @Override
+    public CheckResult checkConfig() {
+        return CheckConfigUtil.checkAllExists(config, "url");
+    }
+
+    @Override
+    public void prepare(SparkEnvironment prepareEnv) {
+    }
+
+    @Override
+    public Dataset<Row> getData(SparkEnvironment env) {
+        SparkSession spark = env.getSparkSession();
+        String url = config.getString(Settings.SOURCE_HTTP_URL);
+        String method = config.hasPath(Settings.SOURCE_HTTP_METHOD) ?
+            config.getString(Settings.SOURCE_HTTP_METHOD) : GET;
+        String header = config.hasPath(Settings.SOURCE_HTTP_HEADER) ?
+            config.getString(Settings.SOURCE_HTTP_HEADER) : "";
+        String requestParams = config.hasPath(Settings.SOURCE_HTTP_REQUEST_PARAMS) ?
+            config.getString(Settings.SOURCE_HTTP_REQUEST_PARAMS) : "";
+        String syncPath = config.hasPath(Settings.SOURCE_HTTP_SYNC_PATH) ?
+            config.getString(Settings.SOURCE_HTTP_SYNC_PATH) : "";
+
+        JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+        Map requestMap = jsonToMap(requestParams);
+        String syncValues = getSyncValues(jsc, syncPath);
+        LOG.info("sync values->{}", syncValues);
+        Map syncMap = jsonToMap(syncValues);
+        if (!syncMap.isEmpty()) {
+            requestMap.putAll(syncMap);
+        }
+
+        HttpClientResult response = new HttpClientResult();
+        try {
+            Map headerMap = jsonToMap(header);
+            if (POST.equals(method)) {
+                response = HttpClientUtils.doPost(url, headerMap, requestMap);
+            } else {
+                response = HttpClientUtils.doGet(url, headerMap, requestMap);
+            }
+        } catch (Exception e) {
+            LOG.error("http call error!", e);
+        }
+
+        LOG.info("http respond code->{}", response.getCode());
+
+        List<String> array = new ArrayList<>();
+        array.add(response.getContent());
+        JavaRDD<String> javaRDD = jsc.parallelize(array);
+        DataFrameReader reader = spark.read().format("json");
+        return reader.json(javaRDD);
+    }
+
+    private String getSyncValues(JavaSparkContext jsc, String syncPath) {
+        if (null == syncPath || syncPath.isEmpty()) {
+            return "";
+        }
+        Configuration hadoopConf = jsc.hadoopConfiguration();
+        List<String> values = new ArrayList<>();
+        try {
+            FileSystem fs = FileSystem.get(hadoopConf);
+            Path path = new Path(syncPath);
+            boolean exists = fs.exists(path);
+            if (exists) {
+                JavaRDD<String> checkPoint = jsc.textFile(syncPath);
+                values = checkPoint.collect();
+
+            }
+        } catch (IOException e) {
+            LOG.error("getSyncValues error, syncPath is {}", syncPath, e);
+        }
+        return values.isEmpty() ? "" : values.iterator().next();
+    }
+
+    private Map jsonToMap(String content) {
+        Map map = new HashMap<>(INITIAL_CAPACITY);
+        if (null == content || content.isEmpty()) {
+            return map;
+        }
+
+        try {
+            return new ObjectMapper().readValue(content, HashMap.class);
+        } catch (IOException e) {
+            LOG.error("{} json to map error!", content, e);

Review Comment:
   You need to add comment when you catch a exception and kill the exception in this catch.



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] tmljob commented on pull request #1678: [Feature#1270][Connectors] add plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
tmljob commented on PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#issuecomment-1094494372

   > @tmljob Please fix unit tests errors, thanks.
   
   This ut has nothing to do with the code submitted by my pr. This is a problem that existed when fork from the dev branch. Do you need me to modify it here?


-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847909895


##########
seatunnel-connectors/seatunnel-connectors-spark/pom.xml:
##########
@@ -52,6 +52,7 @@
         <module>seatunnel-connector-spark-neo4j</module>
         <module>seatunnel-connector-spark-iceberg</module>
         <module>seatunnel-connector-spark-feishu</module>
+		<module>seatunnel-connector-spark-http</module>

Review Comment:
   The format seems mismatching.



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun merged pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun merged PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678


-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] tmljob commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
tmljob commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847865125


##########
seatunnel-connectors/seatunnel-connectors-spark/pom.xml:
##########
@@ -18,40 +18,41 @@
 
 -->
 <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>
-        <groupId>org.apache.seatunnel</groupId>
-        <artifactId>seatunnel-connectors</artifactId>
-        <version>${revision}</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"

Review Comment:
   ok



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] wuchunfu commented on a diff in pull request #1678: [Feature#1270][Connectors] add plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
wuchunfu commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r846149669


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/resources/META-INF/services/org.apache.seatunnel.spark.BaseSparkSource:
##########
@@ -0,0 +1 @@
+org.apache.seatunnel.spark.source.Http

Review Comment:
   @tmljob  Here to add the license header, thanks



-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] ruanwenjun commented on pull request #1678: [Feature#1270][Connectors] add plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
ruanwenjun commented on PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#issuecomment-1094502400

   > > @tmljob Please fix unit tests errors, thanks.
   > 
   > This ut has nothing to do with the code submitted by my pr. This is a problem that existed when fork from the dev branch. Do you need me to modify it here?
   
   @tmljob The ut case should be fixed by upstream, could you please rebase the upstream/dev?


-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] tmljob commented on pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
tmljob commented on PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#issuecomment-1094626986

   > > > @tmljob Please fix unit tests errors, thanks.
   > > 
   > > 
   > > This ut has nothing to do with the code submitted by my pr. This is a problem that existed when fork from the dev branch. Do you need me to modify it here?
   > 
   > @tmljob The ut case should be fixed by upstream, could you please rebase the upstream/dev?
   
   I've synced the latest code from the dev branch, and this ut use case runs fine in my local environment alone.


-- 
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@seatunnel.apache.org

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


[GitHub] [incubator-seatunnel] tmljob commented on a diff in pull request #1678: [Feature#1270][Connectors] add source plugin: seatunnel-connector-spark-http

Posted by GitBox <gi...@apache.org>.
tmljob commented on code in PR #1678:
URL: https://github.com/apache/incubator-seatunnel/pull/1678#discussion_r847017922


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-http/src/main/java/org/apache/seatunnel/spark/source/Http.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.seatunnel.spark.source;
+
+import org.apache.seatunnel.common.config.CheckConfigUtil;
+import org.apache.seatunnel.common.config.CheckResult;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.spark.batch.SparkBatchSource;
+import org.apache.seatunnel.spark.source.constant.Settings;
+import org.apache.seatunnel.spark.source.util.HttpClientResult;
+import org.apache.seatunnel.spark.source.util.HttpClientUtils;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Http extends SparkBatchSource {
+
+    private static final String GET = "GET";
+    private static final String POST = "POST";
+    private static final int INITIAL_CAPACITY = 16;
+
+    private static Logger LOG = LoggerFactory.getLogger(Http.class);
+
+    @Override
+    public CheckResult checkConfig() {
+        return CheckConfigUtil.checkAllExists(config, "url");
+    }
+
+    @Override
+    public void prepare(SparkEnvironment prepareEnv) {
+    }
+
+    @Override
+    public Dataset<Row> getData(SparkEnvironment env) {
+        SparkSession spark = env.getSparkSession();
+        String url = config.getString(Settings.SOURCE_HTTP_URL);
+        String method = config.hasPath(Settings.SOURCE_HTTP_METHOD) ?
+            config.getString(Settings.SOURCE_HTTP_METHOD) : GET;
+        String header = config.hasPath(Settings.SOURCE_HTTP_HEADER) ?
+            config.getString(Settings.SOURCE_HTTP_HEADER) : "";
+        String requestParams = config.hasPath(Settings.SOURCE_HTTP_REQUEST_PARAMS) ?
+            config.getString(Settings.SOURCE_HTTP_REQUEST_PARAMS) : "";
+        String syncPath = config.hasPath(Settings.SOURCE_HTTP_SYNC_PATH) ?
+            config.getString(Settings.SOURCE_HTTP_SYNC_PATH) : "";
+
+        JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
+
+        Map requestMap = jsonToMap(requestParams);
+        String syncValues = getSyncValues(jsc, syncPath);
+        LOG.info("sync values->{}", syncValues);
+        Map syncMap = jsonToMap(syncValues);
+        if (!syncMap.isEmpty()) {
+            requestMap.putAll(syncMap);
+        }
+
+        HttpClientResult response = new HttpClientResult();
+        try {
+            Map headerMap = jsonToMap(header);
+            if (POST.equals(method)) {
+                response = HttpClientUtils.doPost(url, headerMap, requestMap);
+            } else {
+                response = HttpClientUtils.doGet(url, headerMap, requestMap);
+            }
+        } catch (Exception e) {
+            LOG.error("http call error!", e);
+        }
+
+        LOG.info("http respond code->{}", response.getCode());
+
+        List<String> array = new ArrayList<>();
+        array.add(response.getContent());
+        JavaRDD<String> javaRDD = jsc.parallelize(array);
+        DataFrameReader reader = spark.read().format("json");
+        return reader.json(javaRDD);
+    }
+
+    private String getSyncValues(JavaSparkContext jsc, String syncPath) {
+        if (null == syncPath || syncPath.isEmpty()) {
+            return "";
+        }
+        Configuration hadoopConf = jsc.hadoopConfiguration();
+        List<String> values = new ArrayList<>();
+        try {
+            FileSystem fs = FileSystem.get(hadoopConf);
+            Path path = new Path(syncPath);
+            boolean exists = fs.exists(path);
+            if (exists) {
+                JavaRDD<String> checkPoint = jsc.textFile(syncPath);
+                values = checkPoint.collect();
+
+            }
+        } catch (IOException e) {
+            LOG.error("getSyncValues error, syncPath is {}", syncPath, e);
+        }
+        return values.isEmpty() ? "" : values.iterator().next();
+    }
+
+    private Map jsonToMap(String content) {
+        Map map = new HashMap<>(INITIAL_CAPACITY);
+        if (null == content || content.isEmpty()) {
+            return map;
+        }
+
+        try {
+            return new ObjectMapper().readValue(content, HashMap.class);
+        } catch (IOException e) {
+            LOG.error("{} json to map error!", content, e);

Review Comment:
   Catching this exception only records the log, does not handle it, and does not affect the main process.



-- 
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@seatunnel.apache.org

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