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/10/28 12:05:19 UTC

[GitHub] [incubator-seatunnel] hailin0 commented on a diff in pull request #2821: [Feature][Connector-V2][Elasticsearch] Support Elasticsearch source

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