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/09/20 15:04:05 UTC

[GitHub] [incubator-seatunnel] iture123 opened a new pull request, #2821: [Feature][Connector-V2]new connecotor of Elasticsearch source

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

   close #2553 


-- 
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] hailin0 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.DeaultSeaTunnelRowDeserializer;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.ElasticsearchRecord;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.SeaTunnelRowDeserializer;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    private final SeaTunnelRowDeserializer deserializer;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig, SeaTunnelRowType rowTypeInfo) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+        this.deserializer = new DeaultSeaTunnelRowDeserializer(rowTypeInfo);
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+            synchronized (output.getCheckpointLock()) {
+                ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+                while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                    scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                    outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+                }
+            }

Review Comment:
   move queue.poll into synchronized code block
   
   ```suggestion
           if (!splits.isEmpty()) {
               synchronized (output.getCheckpointLock()) {
                   ElasticsearchSourceSplit split = splits.poll();
                   SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
                   ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
                   outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
                       scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
                       outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   }
               }
   ```



-- 
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] hailin0 commented on a diff in pull request #2821: [Feature][Connector-V2]new connecotor of Elasticsearch source

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


##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSink.java:
##########
@@ -45,8 +45,7 @@ public String getPluginName() {
     }
 
     @Override
-    public void prepare(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) throws
-        PrepareFailException {
+    public void prepare(org.apache.seatunnel.shade.com.typesafe.config.Config pluginConfig) throws PrepareFailException {

Review Comment:
   rename `org.apache.seatunnel.shade.com.typesafe.config.Config` to `Config`



##########
docs/en/connector-v2/source/Elasticsearch.md:
##########
@@ -0,0 +1,64 @@
+# Elasticsearch
+
+> Elasticsearch source connector
+
+## Description
+
+Used to read data from Elasticsearch.
+
+support version >= 2.x and < 8.x.
+
+## Key features
+
+- [x] [batch](../../concept/connector-v2-features.md)
+- [x] [stream](../../concept/connector-v2-features.md)
+- [x] [exactly-once](../../concept/connector-v2-features.md)
+- [ ] [schema projection](../../concept/connector-v2-features.md)
+- [ ] [parallelism](../../concept/connector-v2-features.md)
+- [ ] [support user-defined split](../../concept/connector-v2-features.md)
+
+## Options
+
+| name        | type   | required | default value | 
+|-------------|--------| -------- |---------------|
+| hosts       | array  | yes      | -             |
+| username    | string | no       | -             |
+| password    | string | no       | -             |
+| index       | string | yes      | -             |
+| source      | array  | yes      | -             |
+| scroll_time | string | no       | 1m            |
+| scroll_size | int    | no       | 100           |
+
+
+
+### hosts [array]
+Elasticsearch cluster http address, the format is `host:port`, allowing multiple hosts to be specified. Such as `["host1:9200", "host2:9200"]`.
+
+### username [string]
+x-pack username.
+
+### password [string]
+x-pack password.
+
+### index [string]
+Elasticsearch index name, support * fuzzy matching.
+
+### source [array]
+The fields of index.
+You can get the document id by specifying the field `_id`.If sink _id to other index,you need specify an alias for _id due to the Elasticsearch limit.
+
+### scroll_time [String]
+Amount of time Elasticsearch will keep the search context alive for scroll requests.
+
+### scroll_size [int]
+Maximum number of hits to be returned with each Elasticsearch scroll request.
+
+## Examples
+simple

Review Comment:
   Add complex example



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+
+            ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+            outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            }
+        } else if (noMoreSplit) {
+            // signal to the source that we have reached the end of the data.
+            LOG.info("Closed the bounded ELasticsearch source");
+            context.signalNoMoreElement();
+        } else {
+            Thread.sleep(pollNextWaitTime);
+        }
+    }
+
+    private void outputFromScrollResult(ScrollResult scrollResult, List<String> source, Collector<SeaTunnelRow> output) {
+        int sourceSize = source.size();
+        for (Map<String, Object> doc : scrollResult.getDocs()) {
+            SeaTunnelRow seaTunnelRow = new SeaTunnelRow(sourceSize);
+            for (int i = 0; i < sourceSize; i++) {
+                Object value = doc.get(source.get(i));
+                seaTunnelRow.setField(i, String.valueOf(value));

Review Comment:
   `String.valueOf(value)`  Convert all field to string ?
   
   you can mapping elasticsearch datatype to seatunnel datatype ?
   
   reference
   https://github.com/apache/incubator-seatunnel/tree/dev/seatunnel-connectors-v2/connector-iotdb/src/main/java/org/apache/seatunnel/connectors/seatunnel/iotdb/serialize
   
   



##########
seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchSourceToConsoleIT.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.e2e.flink.v2.elasticsearch;
+
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import com.google.common.collect.Lists;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+
+public class ElasticsearchSourceToConsoleIT extends FlinkContainer {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchSourceToConsoleIT.class);
+
+    private ElasticsearchContainer container;
+
+    @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"})
+    @BeforeEach
+    public void startElasticsearchContainer() throws InterruptedException {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));
+        container.start();
+        LOGGER.info("Elasticsearch container started");
+        Thread.sleep(5000L);
+        createIndexDocs();
+    }
+
+    /**
+     * create a index,and bulk some documents
+     */
+    private void createIndexDocs() {
+        EsRestClient esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", "");
+        String requestBody = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" +
+                "{\"name\":\"EbvYoFkXtS\",\"age\":18}\n" +
+                "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" +
+                "{\"name\":\"LjFMprGLJZ\",\"age\":19}\n" +
+                "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}}\n" +
+                "{\"name\":\"uJTtAVuSyI\",\"age\":20}\n";
+        esRestClient.bulk(requestBody);
+        try {
+            esRestClient.close();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Test
+    public void testElasticsearchSourceToConsoleSink() throws IOException, InterruptedException {
+        Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_to_console.conf");
+        Assertions.assertEquals(0, execResult.getExitCode());

Review Comment:
   validate data rows & values & types when e2e execute ends
   
   
   
   reference
   https://github.com/apache/incubator-seatunnel/blob/dev/seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-mongodb-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/mongodb/MongodbIT.java#L106
   
   



-- 
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] iture123 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+
+            ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+            outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            }

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] hailin0 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
seatunnel-connectors-v2/connector-elasticsearch/pom.xml:
##########
@@ -42,15 +39,21 @@
             <version>${elasticsearch-rest-client.version}</version>
         </dependency>
         <dependency>
-            <groupId>com.fasterxml.jackson.core</groupId>
-            <artifactId>jackson-databind</artifactId>
-            <version>${jackson.databind.version}</version>
+            <groupId>org.apache.seatunnel</groupId>
+            <artifactId>connector-common</artifactId>
+            <version>${project.version}</version>
+            <scope>compile</scope>
         </dependency>
         <dependency>
-            <groupId>com.fasterxml.jackson.datatype</groupId>
-            <artifactId>jackson-datatype-jsr310</artifactId>
-            <version>${jackson-datatype-jsr310.version}</version>
+            <groupId>org.apache.seatunnel</groupId>
+            <artifactId>seatunnel-api</artifactId>
+            <version>${project.version}</version>
+            <scope>compile</scope>
+        </dependency>

Review Comment:
   remove seatunnel-api
   
   
   https://github.com/apache/incubator-seatunnel/blob/dev/seatunnel-connectors-v2/pom.xml#L64



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.DeaultSeaTunnelRowDeserializer;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.ElasticsearchRecord;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.serialize.source.SeaTunnelRowDeserializer;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    private final SeaTunnelRowDeserializer deserializer;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig, SeaTunnelRowType rowTypeInfo) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+        this.deserializer = new DeaultSeaTunnelRowDeserializer(rowTypeInfo);
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+            synchronized (output.getCheckpointLock()) {
+                ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+                while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                    scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                    outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+                }
+            }

Review Comment:
   move queue.poll into synchronized code block
   
   ```suggestion
           if (!pendingSplits.isEmpty()) {
               synchronized (output.getCheckpointLock()) {
                   ElasticsearchSourceSplit split = splits.poll();
                   SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
                   ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
                   outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
                       scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
                       outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   }
               }
   ```



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/sink/ElasticsearchSinkWriter.java:
##########
@@ -43,43 +44,31 @@
 /**
  * ElasticsearchSinkWriter is a sink writer that will write {@link SeaTunnelRow} to Elasticsearch.
  */
-@Slf4j

Review Comment:
   why delete this?



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfigDeaultConstant;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchSourceSplitEnumerator implements SourceSplitEnumerator<ElasticsearchSourceSplit, ElasticsearchSourceState> {
+
+    private SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    private final Object stateLock = new Object();
+
+    private Map<Integer, List<ElasticsearchSourceSplit>> pendingSplit;
+
+    private List<String> source;
+
+    private volatile boolean shouldEnumerate;
+
+    public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context, Config pluginConfig, List<String> source) {
+        this(context, null, pluginConfig, source);
+    }
+
+    public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context, ElasticsearchSourceState sourceState, Config pluginConfig, List<String> source) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+        this.pendingSplit = new HashMap<>();
+        this.shouldEnumerate = sourceState == null;
+        if (sourceState != null) {
+            this.shouldEnumerate = sourceState.isShouldEnumerate();
+            this.pendingSplit.putAll(sourceState.getPendingSplit());
+        }
+        this.source = source;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(pluginConfig);
+    }
+
+    @Override
+    public void run() {
+        Set<Integer> readers = context.registeredReaders();
+        if (shouldEnumerate) {
+            List<ElasticsearchSourceSplit> newSplits = getElasticsearchSplit();
+
+            synchronized (stateLock) {
+                addPendingSplit(newSplits);
+                shouldEnumerate = false;
+            }
+
+            assignSplit(readers);
+        }
+
+        log.debug("No more splits to assign." +
+                " Sending NoMoreSplitsEvent to reader {}.", readers);
+        readers.forEach(context::signalNoMoreSplits);
+    }
+
+    private void addPendingSplit(Collection<ElasticsearchSourceSplit> splits) {
+        int readerCount = context.currentParallelism();
+        for (ElasticsearchSourceSplit split : splits) {
+            int ownerReader = getSplitOwner(split.splitId(), readerCount);
+            log.info("Assigning {} to {} reader.", split, ownerReader);
+            pendingSplit.computeIfAbsent(ownerReader, r -> new ArrayList<>())
+                    .add(split);
+        }
+    }
+
+    private static int getSplitOwner(String tp, int numReaders) {
+        return (tp.hashCode() & Integer.MAX_VALUE) % numReaders;
+    }
+
+    private void assignSplit(Collection<Integer> readers) {
+        log.debug("Assign pendingSplits to readers {}", readers);
+
+        for (int reader : readers) {
+            List<ElasticsearchSourceSplit> assignmentForReader = pendingSplit.remove(reader);
+            if (assignmentForReader != null && !assignmentForReader.isEmpty()) {
+                log.info("Assign splits {} to reader {}",
+                        assignmentForReader, reader);
+                try {
+                    context.assignSplit(reader, assignmentForReader);
+                } catch (Exception e) {
+                    log.error("Failed to assign splits {} to reader {}",
+                            assignmentForReader, reader, e);
+                    pendingSplit.put(reader, assignmentForReader);
+                }
+            }
+        }
+    }
+
+    private List<ElasticsearchSourceSplit> getElasticsearchSplit() {
+        List<ElasticsearchSourceSplit> splits = new ArrayList<>();
+        String scrolllTime = SourceConfigDeaultConstant.SCROLLL_TIME;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_TIME)) {
+            scrolllTime = pluginConfig.getString(SourceConfig.SCROLL_TIME);
+        }
+        int scrollSize = SourceConfigDeaultConstant.SCROLLL_SIZE;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_SIZE)) {
+            scrollSize = pluginConfig.getInt(SourceConfig.SCROLL_SIZE);
+        }
+
+        List<IndexDocsCount> indexDocsCounts = esRestClient.getIndexDocsCount(pluginConfig.getString(SourceConfig.INDEX));
+        indexDocsCounts = indexDocsCounts.stream().filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0)
+                .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)).collect(Collectors.toList());
+        for (IndexDocsCount indexDocsCount : indexDocsCounts) {
+            splits.add(new ElasticsearchSourceSplit(String.valueOf(indexDocsCount.getIndex().hashCode()), new SourceIndexInfo(indexDocsCount.getIndex(), source, scrolllTime, scrollSize)));
+        }
+        return splits;
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void addSplitsBack(List<ElasticsearchSourceSplit> splits, int subtaskId) {
+
+    }

Review Comment:
   ```suggestion
       public void addSplitsBack(List<ElasticsearchSourceSplit> splits, int subtaskId) {
           if (!splits.isEmpty()) {
               addPendingSplit(splits);
               assignSplit(Collections.singletonList(subtaskId));
           }
       }
   ```



##########
seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.e2e.flink.v2.elasticsearch;
+
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends FlinkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));
+        container.start();
+        LOGGER.info("Elasticsearch container started");
+        esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", "");
+        testDataset = generateTestDataSet();
+        createIndexDocs();
+    }
+
+    /**
+     * create a index,and bulk some documents
+     */
+    private void createIndexDocs() {
+        StringBuilder requestBody = new StringBuilder();
+        Map<String, String> indexInner = new HashMap<>();
+        indexInner.put("_index", "st");
+        indexInner.put("_index", "st");
+
+        Map<String, Map<String, String>> indexParam = new HashMap<>();
+        indexParam.put("index", indexInner);
+        String indexHeader = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}\n";
+        for (int i = 0; i < testDataset.size(); i++) {
+            String row = testDataset.get(i);
+            requestBody.append(indexHeader);
+            requestBody.append(row);
+            requestBody.append("\n");
+        }
+        esRestClient.bulk(requestBody.toString());
+    }
+
+    @Test
+    public void testElasticsearch() throws IOException, InterruptedException {
+        Container.ExecResult execResult = executeSeaTunnelFlinkJob("/elasticsearch/elasticsearch_source_and_sink.conf");
+        Assertions.assertEquals(0, execResult.getExitCode());
+        List<String> sinData = readSinkData();
+        Assertions.assertIterableEquals(
+                testDataset,
+                sinData
+        );
+    }
+
+    private List<String> generateTestDataSet() throws JsonProcessingException, UnknownHostException {
+        String[] fiels = new String[]{

Review Comment:
   ```suggestion
           String[] fields = new String[]{
   ```



##########
seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.e2e.flink.v2.elasticsearch;
+
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends FlinkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));

Review Comment:
   ```suggestion
                   .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger("elasticsearch:6.8.23")));
   ```



##########
seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.e2e.spark.v2.elasticsearch;
+
+import lombok.SneakyThrows;
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.spark.SparkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.shaded.org.apache.commons.lang3.ThreadUtils;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends SparkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));
+        container.start();
+        LOGGER.info("Elasticsearch container started");
+        esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", "");
+        testDataset = generateTestDataSet();
+        createIndexDocs();
+
+    }
+
+    /**
+     * create a index,and bulk some documents
+     */
+    private void createIndexDocs() {
+        StringBuilder requestBody = new StringBuilder();
+        Map<String, String> indexInner = new HashMap<>();
+        indexInner.put("_index", "st");
+        indexInner.put("_index", "st");
+
+        Map<String, Map<String, String>> indexParam = new HashMap<>();
+        indexParam.put("index", indexInner);
+        String indexHeader = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}\n";
+        for (int i = 0; i < testDataset.size(); i++) {
+            String row = testDataset.get(i);
+            requestBody.append(indexHeader);
+            requestBody.append(row);
+            requestBody.append("\n");
+        }
+        esRestClient.bulk(requestBody.toString());
+    }
+
+    @Test
+    public void testElasticsearch() throws IOException, InterruptedException {
+        Container.ExecResult execResult = executeSeaTunnelSparkJob("/elasticsearch/elasticsearch_source_and_sink.conf");
+        Assertions.assertEquals(0, execResult.getExitCode());
+        List<String> sinData = readSinkData();
+        Assertions.assertIterableEquals(
+                testDataset,
+                sinData
+        );
+    }
+
+    private List<String> generateTestDataSet() throws JsonProcessingException, UnknownHostException {
+        String[] fiels = new String[]{

Review Comment:
   ```suggestion
           String[] fields = new String[]{
   ```



##########
seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.e2e.spark.v2.elasticsearch;
+
+import lombok.SneakyThrows;
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.spark.SparkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.shaded.org.apache.commons.lang3.ThreadUtils;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends SparkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {

Review Comment:
   ```suggestion
       public void startElasticsearchContainer() throws Exception {
   ```



##########
seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.e2e.flink.v2.elasticsearch;
+
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends FlinkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {

Review Comment:
   ```suggestion
       public void startElasticsearchContainer() throws Exception {
   ```



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfigDeaultConstant;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchSourceSplitEnumerator implements SourceSplitEnumerator<ElasticsearchSourceSplit, ElasticsearchSourceState> {
+
+    private SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    private final Object stateLock = new Object();
+
+    private Map<Integer, List<ElasticsearchSourceSplit>> pendingSplit;
+
+    private List<String> source;
+
+    private volatile boolean shouldEnumerate;
+
+    public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context, Config pluginConfig, List<String> source) {
+        this(context, null, pluginConfig, source);
+    }
+
+    public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context<ElasticsearchSourceSplit> context, ElasticsearchSourceState sourceState, Config pluginConfig, List<String> source) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+        this.pendingSplit = new HashMap<>();
+        this.shouldEnumerate = sourceState == null;
+        if (sourceState != null) {
+            this.shouldEnumerate = sourceState.isShouldEnumerate();
+            this.pendingSplit.putAll(sourceState.getPendingSplit());
+        }
+        this.source = source;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(pluginConfig);
+    }
+
+    @Override
+    public void run() {
+        Set<Integer> readers = context.registeredReaders();
+        if (shouldEnumerate) {
+            List<ElasticsearchSourceSplit> newSplits = getElasticsearchSplit();
+
+            synchronized (stateLock) {
+                addPendingSplit(newSplits);
+                shouldEnumerate = false;
+            }
+
+            assignSplit(readers);
+        }
+
+        log.debug("No more splits to assign." +
+                " Sending NoMoreSplitsEvent to reader {}.", readers);
+        readers.forEach(context::signalNoMoreSplits);
+    }
+
+    private void addPendingSplit(Collection<ElasticsearchSourceSplit> splits) {
+        int readerCount = context.currentParallelism();
+        for (ElasticsearchSourceSplit split : splits) {
+            int ownerReader = getSplitOwner(split.splitId(), readerCount);
+            log.info("Assigning {} to {} reader.", split, ownerReader);
+            pendingSplit.computeIfAbsent(ownerReader, r -> new ArrayList<>())
+                    .add(split);
+        }
+    }
+
+    private static int getSplitOwner(String tp, int numReaders) {
+        return (tp.hashCode() & Integer.MAX_VALUE) % numReaders;
+    }
+
+    private void assignSplit(Collection<Integer> readers) {
+        log.debug("Assign pendingSplits to readers {}", readers);
+
+        for (int reader : readers) {
+            List<ElasticsearchSourceSplit> assignmentForReader = pendingSplit.remove(reader);
+            if (assignmentForReader != null && !assignmentForReader.isEmpty()) {
+                log.info("Assign splits {} to reader {}",
+                        assignmentForReader, reader);
+                try {
+                    context.assignSplit(reader, assignmentForReader);
+                } catch (Exception e) {
+                    log.error("Failed to assign splits {} to reader {}",
+                            assignmentForReader, reader, e);
+                    pendingSplit.put(reader, assignmentForReader);
+                }
+            }
+        }
+    }
+
+    private List<ElasticsearchSourceSplit> getElasticsearchSplit() {
+        List<ElasticsearchSourceSplit> splits = new ArrayList<>();
+        String scrolllTime = SourceConfigDeaultConstant.SCROLLL_TIME;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_TIME)) {
+            scrolllTime = pluginConfig.getString(SourceConfig.SCROLL_TIME);
+        }
+        int scrollSize = SourceConfigDeaultConstant.SCROLLL_SIZE;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_SIZE)) {
+            scrollSize = pluginConfig.getInt(SourceConfig.SCROLL_SIZE);
+        }
+
+        List<IndexDocsCount> indexDocsCounts = esRestClient.getIndexDocsCount(pluginConfig.getString(SourceConfig.INDEX));
+        indexDocsCounts = indexDocsCounts.stream().filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0)
+                .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)).collect(Collectors.toList());
+        for (IndexDocsCount indexDocsCount : indexDocsCounts) {
+            splits.add(new ElasticsearchSourceSplit(String.valueOf(indexDocsCount.getIndex().hashCode()), new SourceIndexInfo(indexDocsCount.getIndex(), source, scrolllTime, scrollSize)));
+        }
+        return splits;
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void addSplitsBack(List<ElasticsearchSourceSplit> splits, int subtaskId) {
+
+    }
+
+    @Override
+    public int currentUnassignedSplitSize() {
+        return 0;
+    }

Review Comment:
   ```suggestion
       public int currentUnassignedSplitSize() {
           return pendingSplit.size();
       }
   ```



##########
seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.e2e.spark.v2.elasticsearch;
+
+import lombok.SneakyThrows;
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.spark.SparkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.shaded.org.apache.commons.lang3.ThreadUtils;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends SparkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));

Review Comment:
   ```suggestion
                   .withLogConsumer(new Slf4jLogConsumer(DockerLoggerFactory.getLogger("elasticsearch:6.8.23")));
   ```



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplit.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import lombok.ToString;
+
+@ToString
+public class ElasticsearchSourceSplit implements SourceSplit {
+
+    private static final long serialVersionUID = -1L;
+
+    private String splitId;
+
+    private SourceIndexInfo sourceIndexInfo;
+
+    public ElasticsearchSourceSplit(String splitId, SourceIndexInfo sourceIndexInfo) {
+        this.splitId = splitId;
+        this.sourceIndexInfo = sourceIndexInfo;
+    }
+
+    public SourceIndexInfo getSourceIndexInfo() {
+        return sourceIndexInfo;
+    }
+
+    @Override
+    public String splitId() {
+        return splitId;
+    }

Review Comment:
   Use lombok  annotation



-- 
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] hailin0 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
docs/en/connector-v2/source/Elasticsearch.md:
##########
@@ -0,0 +1,64 @@
+# Elasticsearch
+
+> Elasticsearch source connector
+
+## Description
+
+Used to read data from Elasticsearch.
+
+support version >= 2.x and < 8.x.
+
+## Key features
+
+- [x] [batch](../../concept/connector-v2-features.md)
+- [x] [stream](../../concept/connector-v2-features.md)
+- [x] [exactly-once](../../concept/connector-v2-features.md)

Review Comment:
   How to restore split(exactly-once)?
   
   reference restore split
   https://github.com/apache/incubator-seatunnel/pull/2917



##########
docs/en/connector-v2/source/Elasticsearch.md:
##########
@@ -0,0 +1,64 @@
+# Elasticsearch
+
+> Elasticsearch source connector
+
+## Description
+
+Used to read data from Elasticsearch.
+
+support version >= 2.x and < 8.x.
+
+## Key features
+
+- [x] [batch](../../concept/connector-v2-features.md)
+- [x] [stream](../../concept/connector-v2-features.md)

Review Comment:
   support stream read?



-- 
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] ashulin commented on a diff in pull request #2821: [Feature][Connector-V2]new connecotor of Elasticsearch source

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


##########
seatunnel-e2e/seatunnel-flink-connector-v2-e2e/connector-elasticsearch-flink-e2e/src/test/java/org/apache/seatunnel/e2e/flink/v2/elasticsearch/FakeSourceToElasticsearchIT.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.e2e.flink.v2.elasticsearch;
+
+import org.apache.seatunnel.e2e.flink.FlinkContainer;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+
+public class FakeSourceToElasticsearchIT extends FlinkContainer {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FakeSourceToElasticsearchIT.class);
+
+    private ElasticsearchContainer container;
+
+    @SuppressWarnings({"checkstyle:MagicNumber", "checkstyle:Indentation"})

Review Comment:
   ```suggestion
   ```
   IT will automatically ignore MagicNumber, please do not ignore the indentation.



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceSplitEnumerator.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.SourceSplitEnumerator;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfig;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.config.source.SourceConfigDeaultConstant;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.IndexDocsCount;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class ElasticsearchSourceSplitEnumerator implements SourceSplitEnumerator<ElasticsearchSourceSplit, ElasticsearchSourceState> {
+
+    private SourceSplitEnumerator.Context<ElasticsearchSourceSplit> enumeratorContext;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    public ElasticsearchSourceSplitEnumerator(SourceSplitEnumerator.Context<ElasticsearchSourceSplit> enumeratorContext, Config pluginConfig) {
+        this.enumeratorContext = enumeratorContext;
+        this.pluginConfig = pluginConfig;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(pluginConfig);
+    }
+
+    @Override
+    public void run() throws Exception {
+
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void addSplitsBack(List<ElasticsearchSourceSplit> splits, int subtaskId) {
+
+    }
+
+    @Override
+    public int currentUnassignedSplitSize() {
+        return 0;
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId) {
+
+    }
+
+    @Override
+    public void registerReader(int subtaskId) {
+        String scrolllTime = SourceConfigDeaultConstant.SCROLLL_TIME;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_TIME)) {
+            scrolllTime = pluginConfig.getString(SourceConfig.SCROLL_TIME);
+        }
+        int scrollSize = SourceConfigDeaultConstant.SCROLLL_SIZE;
+        if (pluginConfig.hasPath(SourceConfig.SCROLL_SIZE)) {
+            scrollSize = pluginConfig.getInt(SourceConfig.SCROLL_SIZE);
+        }
+
+        List<IndexDocsCount> indexDocsCounts = esRestClient.getIndexDocsCount(pluginConfig.getString(SourceConfig.INDEX));
+        indexDocsCounts = indexDocsCounts.stream().filter(x -> x.getDocsCount() != null && x.getDocsCount() > 0)
+                .sorted(Comparator.comparingLong(IndexDocsCount::getDocsCount)).collect(Collectors.toList());
+        List<ElasticsearchSourceSplit> splits = new ArrayList<>();
+        int parallelism = enumeratorContext.currentParallelism();

Review Comment:
   `EnumeratorContext.currentParallelism()` is Runtime Method, Use it in `SplitEnumerator#run()`



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+
+            ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+            outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            }

Review Comment:
   ```suggestion
               synchronized (output.getCheckpointLock()) {
                   ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
                   outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
                       scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
                       outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
                   }
               }
   ```



##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/dto/source/SourceIndexInfo.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.dto.source;
+
+import java.io.Serializable;
+import java.util.List;
+
+public class SourceIndexInfo implements Serializable {

Review Comment:
   ```suggestion
   @Data
   @AllArgsConstructor
   public class SourceIndexInfo implements Serializable {
   ```
   Please use lombok.



-- 
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] EricJoy2048 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
docs/en/connector-v2/source/Elasticsearch.md:
##########
@@ -0,0 +1,89 @@
+# Elasticsearch
+
+> Elasticsearch source connector
+
+## Description
+
+Used to read data from Elasticsearch.
+
+support version >= 2.x and < 8.x.
+
+## Key features
+
+- [x] [batch](../../concept/connector-v2-features.md)
+- [ ] [stream](../../concept/connector-v2-features.md)
+- [ ] [exactly-once](../../concept/connector-v2-features.md)
+- [x] [schema projection](../../concept/connector-v2-features.md)
+- [ ] [parallelism](../../concept/connector-v2-features.md)
+- [ ] [support user-defined split](../../concept/connector-v2-features.md)
+
+## Options
+
+| name        | type   | required | default value | 
+|-------------|--------| -------- |---------------|
+| hosts       | array  | yes      | -             |
+| username    | string | no       | -             |
+| password    | string | no       | -             |
+| index       | string | yes      | -             |
+| source      | array  | yes      | -             |
+| scroll_time | string | no       | 1m            |
+| scroll_size | int    | no       | 100           |
+
+
+
+### hosts [array]
+Elasticsearch cluster http address, the format is `host:port`, allowing multiple hosts to be specified. Such as `["host1:9200", "host2:9200"]`.
+
+### username [string]
+x-pack username.
+
+### password [string]
+x-pack password.
+
+### index [string]
+Elasticsearch index name, support * fuzzy matching.
+
+### source [array]
+The fields of index.
+You can get the document id by specifying the field `_id`.If sink _id to other index,you need specify an alias for _id due to the Elasticsearch limit.
+
+### scroll_time [String]
+Amount of time Elasticsearch will keep the search context alive for scroll requests.
+
+### scroll_size [int]
+Maximum number of hits to be returned with each Elasticsearch scroll request.
+
+## Examples
+simple
+```hocon
+Elasticsearch {
+    hosts = ["localhost:9200"]
+    index = "seatunnel-*"
+    source = ["_id","name","age"]
+}
+```
+complex
+```hocon
+Elasticsearch {
+    hosts = ["elasticsearch:9200"]
+    index = "st_index"
+    schema = {
+        fields {
+            c_map = "map<string, tinyint>"
+            c_array = "array<tinyint>"
+            c_string = string
+            c_boolean = boolean
+            c_tinyint = tinyint
+            c_smallint = smallint
+            c_int = int
+            c_bigint = bigint
+            c_float = float
+            c_double = double
+            c_decimal = "decimal(2, 1)"
+            c_bytes = bytes
+            c_date = date
+            c_timestamp = timestamp
+        }
+    }
+}
+```

Review Comment:
   Please add `Change log` reference https://github.com/apache/incubator-seatunnel/blob/dev/docs/en/connector-v2/source/SftpFile.md



-- 
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] iture123 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
seatunnel-connectors-v2/connector-elasticsearch/src/main/java/org/apache/seatunnel/connectors/seatunnel/elasticsearch/source/ElasticsearchSourceReader.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.connectors.seatunnel.elasticsearch.source;
+
+import org.apache.seatunnel.api.source.Collector;
+import org.apache.seatunnel.api.source.SourceReader;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.SourceIndexInfo;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class ElasticsearchSourceReader implements SourceReader<SeaTunnelRow, ElasticsearchSourceSplit> {
+
+    protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSourceReader.class);
+
+    SourceReader.Context context;
+
+    private Config pluginConfig;
+
+    private EsRestClient esRestClient;
+
+    Deque<ElasticsearchSourceSplit> splits = new LinkedList<>();
+    boolean noMoreSplit;
+
+    private final long pollNextWaitTime = 1000L;
+
+    public ElasticsearchSourceReader(SourceReader.Context context, Config pluginConfig) {
+        this.context = context;
+        this.pluginConfig = pluginConfig;
+    }
+
+    @Override
+    public void open() {
+        esRestClient = EsRestClient.createInstance(this.pluginConfig);
+    }
+
+    @Override
+    public void close() throws IOException {
+        esRestClient.close();
+    }
+
+    @Override
+    public void pollNext(Collector<SeaTunnelRow> output) throws Exception {
+        ElasticsearchSourceSplit split = splits.poll();
+        if (null != split) {
+            SourceIndexInfo sourceIndexInfo = split.getSourceIndexInfo();
+
+            ScrollResult scrollResult = esRestClient.searchByScroll(sourceIndexInfo.getIndex(), sourceIndexInfo.getSource(), sourceIndexInfo.getScrollTime(), sourceIndexInfo.getScrollSize());
+            outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            while (scrollResult.getDocs() != null && scrollResult.getDocs().size() > 0) {
+                scrollResult = esRestClient.searchWithScrollId(scrollResult.getScrollId(), sourceIndexInfo.getScrollTime());
+                outputFromScrollResult(scrollResult, sourceIndexInfo.getSource(), output);
+            }
+        } else if (noMoreSplit) {
+            // signal to the source that we have reached the end of the data.
+            LOG.info("Closed the bounded ELasticsearch source");
+            context.signalNoMoreElement();
+        } else {
+            Thread.sleep(pollNextWaitTime);
+        }
+    }
+
+    private void outputFromScrollResult(ScrollResult scrollResult, List<String> source, Collector<SeaTunnelRow> output) {
+        int sourceSize = source.size();
+        for (Map<String, Object> doc : scrollResult.getDocs()) {
+            SeaTunnelRow seaTunnelRow = new SeaTunnelRow(sourceSize);
+            for (int i = 0; i < sourceSize; i++) {
+                Object value = doc.get(source.get(i));
+                seaTunnelRow.setField(i, String.valueOf(value));

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] iture123 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
docs/en/connector-v2/source/Elasticsearch.md:
##########
@@ -0,0 +1,64 @@
+# Elasticsearch
+
+> Elasticsearch source connector
+
+## Description
+
+Used to read data from Elasticsearch.
+
+support version >= 2.x and < 8.x.
+
+## Key features
+
+- [x] [batch](../../concept/connector-v2-features.md)
+- [x] [stream](../../concept/connector-v2-features.md)

Review Comment:
   Don't support.I will revise.



-- 
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] EricJoy2048 merged pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


-- 
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] iture123 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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


##########
seatunnel-e2e/seatunnel-spark-connector-v2-e2e/connector-elasticsearch-spark-e2e/src/test/java/org/apache/seatunnel/e2e/spark/v2/elasticsearch/ElasticsearchIT.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.e2e.spark.v2.elasticsearch;
+
+import lombok.SneakyThrows;
+import org.apache.seatunnel.common.utils.JsonUtils;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.client.EsRestClient;
+import org.apache.seatunnel.connectors.seatunnel.elasticsearch.dto.source.ScrollResult;
+import org.apache.seatunnel.e2e.spark.SparkContainer;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.Container;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.shaded.org.apache.commons.lang3.ThreadUtils;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.net.UnknownHostException;
+import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@Slf4j
+public class ElasticsearchIT extends SparkContainer {
+
+    private List<String> testDataset;
+
+    private ElasticsearchContainer container;
+
+    private EsRestClient esRestClient;
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ElasticsearchIT.class);
+
+    @BeforeEach
+    public void startMongoContainer() throws Exception {
+        container = new ElasticsearchContainer(DockerImageName.parse("elasticsearch:6.8.23").asCompatibleSubstituteFor("docker.elastic.co/elasticsearch/elasticsearch"))
+                .withNetwork(NETWORK)
+                .withNetworkAliases("elasticsearch")
+                .withLogConsumer(new Slf4jLogConsumer(LOGGER));
+        container.start();
+        LOGGER.info("Elasticsearch container started");
+        esRestClient = EsRestClient.createInstance(Lists.newArrayList(container.getHttpHostAddress()), "", "");
+        testDataset = generateTestDataSet();
+        createIndexDocs();
+
+    }
+
+    /**
+     * create a index,and bulk some documents
+     */
+    private void createIndexDocs() {
+        StringBuilder requestBody = new StringBuilder();
+        Map<String, String> indexInner = new HashMap<>();
+        indexInner.put("_index", "st");
+        indexInner.put("_index", "st");
+
+        Map<String, Map<String, String>> indexParam = new HashMap<>();
+        indexParam.put("index", indexInner);
+        String indexHeader = "{\"index\":{\"_index\":\"st_index\",\"_type\":\"st\"}\n";
+        for (int i = 0; i < testDataset.size(); i++) {
+            String row = testDataset.get(i);
+            requestBody.append(indexHeader);
+            requestBody.append(row);
+            requestBody.append("\n");
+        }
+        esRestClient.bulk(requestBody.toString());
+    }
+
+    @Test
+    public void testElasticsearch() throws IOException, InterruptedException {
+        Container.ExecResult execResult = executeSeaTunnelSparkJob("/elasticsearch/elasticsearch_source_and_sink.conf");
+        Assertions.assertEquals(0, execResult.getExitCode());
+        List<String> sinData = readSinkData();
+        Assertions.assertIterableEquals(
+                testDataset,
+                sinData
+        );
+    }
+
+    private List<String> generateTestDataSet() throws JsonProcessingException, UnknownHostException {
+        String[] fiels = new String[]{

Review Comment:
   All your suggestions is improved.Thinks.



-- 
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] EricJoy2048 commented on pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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

   I will merge this pr and please add I will add a pr to change log reference https://github.com/apache/incubator-seatunnel/blob/dev/docs/en/connector-v2/source/SftpFile.md 


-- 
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