You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/04/08 11:49:46 UTC

[GitHub] [flink] fapaul commented on a diff in pull request #19405: [FLINK-27066] Reintroduce e2e tests in ES as Java tests.

fapaul commented on code in PR #19405:
URL: https://github.com/apache/flink/pull/19405#discussion_r846017025


##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/pom.xml:
##########
@@ -0,0 +1,158 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-end-to-end-tests</artifactId>
+		<version>1.16-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-end-to-end-tests-elasticsearch6</artifactId>
+	<name>Flink : E2E Tests : Elasticsearch 6 Java</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<elasticsearch.version>6.8.20</elasticsearch.version>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-elasticsearch6</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>1.16-SNAPSHOT</version>

Review Comment:
   I guess all `1.16-SNAPTHOT` should be `${project.version}`



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/main/java/com/apache/flink/streaming/tests/TupleC2.java:
##########
@@ -0,0 +1,30 @@
+package com.apache.flink.streaming.tests;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+
+/** A {@link Comparable} implementation of {@link Tuple2}. */
+public class TupleC2<T0 extends Comparable<? super T0>, T1 extends Comparable<? super T1>>

Review Comment:
   I would much rather have a dedicated class for the test results than introduce a new `Tuple` class.



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/pom.xml:
##########
@@ -0,0 +1,158 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-end-to-end-tests</artifactId>
+		<version>1.16-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-end-to-end-tests-elasticsearch6</artifactId>
+	<name>Flink : E2E Tests : Elasticsearch 6 Java</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<elasticsearch.version>6.8.20</elasticsearch.version>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-elasticsearch6</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>1.16-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-end-to-end-tests-common</artifactId>
+			<version>1.16-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-end-to-end-tests-common-elasticsearch</artifactId>
+			<version>1.16-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>elasticsearch</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>Elasticsearch6SinkExample</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<finalName>elasticsearch6-end-to-end-test</finalName>
+							<outputDirectory>dependencies</outputDirectory>
+							<artifactSet>
+								<excludes>
+									<exclude>com.google.code.findbugs:jsr305</exclude>
+								</excludes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>*:*</artifact>
+									<excludes>
+										<exclude>META-INF/*.SF</exclude>
+										<exclude>META-INF/*.DSA</exclude>
+										<exclude>META-INF/*.RSA</exclude>
+									</excludes>
+								</filter>
+							</filters>
+							<transformers>
+								<transformer
+									implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+									<mainClass>
+										org.apache.flink.streaming.tests.Elasticsearch6SinkExample

Review Comment:
   Is the `Elasticsearch6SinkExample` used anywhere still? If not we can remove most of the complexity of the pom.



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6DataReader.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/** Elasticsearch data reader. */
+public class Elasticsearch6DataReader implements ExternalSystemDataReader<TupleC2<String, String>> {
+    private final RestHighLevelClient client;
+    private final String indexName;
+    private final int pageLength;
+    private int from = 0;
+
+    public Elasticsearch6DataReader(RestHighLevelClient client, String indexName, int pageLength) {
+        this.client = client;
+        this.indexName = indexName;
+        this.pageLength = pageLength;
+    }
+
+    @Override
+    public List<TupleC2<String, String>> poll(Duration timeout) {
+        List<TupleC2<String, String>> result = new ArrayList<>();
+        try {
+            SearchResponse response =
+                    client.search(
+                            new SearchRequest(indexName)
+                                    .source(
+                                            new SearchSourceBuilder()
+                                                    .query(QueryBuilders.matchAllQuery())
+                                                    .from(from)
+                                                    .size(pageLength)
+                                                    .trackTotalHits(true)),
+                            RequestOptions.DEFAULT);
+            SearchHit[] searchHits = response.getHits().getHits();
+            for (SearchHit searchHit : searchHits) {
+                TupleC2<String, String> hit =
+                        TupleC2.of(
+                                searchHit.getId(),
+                                searchHit.getSourceAsMap().get("data").toString());
+                result.add(hit);
+            }
+            from += searchHits.length;
+            return result;
+        } catch (IOException e) {
+            return Collections.emptyList();
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        if (client != null) {

Review Comment:
   Can the client be null?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkE2ECase.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.flink.FlinkContainerTestEnvironment;
+import org.apache.flink.util.DockerImageVersions;
+
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.util.Arrays;
+
+/** End to end test for Elasticsearch7Sink based on connector testing framework. */
+@SuppressWarnings("unused")
+public class Elasticsearch6SinkE2ECase extends SinkTestSuiteBase<TupleC2<String, String>> {
+    private static final String ELASTICSEARCH_HOSTNAME = "elasticsearch";
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    // Defines TestEnvironment
+    @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment(1, 6);
+
+    // Defines ConnectorExternalSystem
+    @TestExternalSystem
+    DefaultContainerizedExternalSystem<ElasticsearchContainer> elasticsearch =
+            DefaultContainerizedExternalSystem.builder()
+                    .fromContainer(
+                            new ElasticsearchContainer(
+                                            DockerImageName.parse(
+                                                    DockerImageVersions.ELASTICSEARCH_7))

Review Comment:
   ```suggestion
                                                       DockerImageVersions.ELASTICSEARCH_6))
   ```



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6DataReader.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/** Elasticsearch data reader. */
+public class Elasticsearch6DataReader implements ExternalSystemDataReader<TupleC2<String, String>> {
+    private final RestHighLevelClient client;
+    private final String indexName;
+    private final int pageLength;
+    private int from = 0;
+
+    public Elasticsearch6DataReader(RestHighLevelClient client, String indexName, int pageLength) {
+        this.client = client;
+        this.indexName = indexName;
+        this.pageLength = pageLength;

Review Comment:
   please add `checkNotNull` for the non nullable parameters.



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkExternalContext.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.http.HttpHost;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+class Elasticsearch6SinkExternalContext
+        implements DataStreamSinkV2ExternalContext<TupleC2<String, String>> {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(Elasticsearch6SinkExternalContext.class);
+
+    private static final String INDEX_NAME_PREFIX = "es-index";
+    private static final int RANDOM_STRING_MAX_LENGTH = 50;
+    private static final int NUM_RECORDS_UPPER_BOUND = 500;
+    private static final int NUM_RECORDS_LOWER_BOUND = 100;
+    private static final int PAGE_LENGTH = 100;
+
+    protected final String indexName;
+
+    private final String addressInternal;
+    private final List<URL> connectorJarPaths;
+    private final RestHighLevelClient client;
+
+    Elasticsearch6SinkExternalContext(
+            String addressExternal, String addressInternal, List<URL> connectorJarPaths) {
+        this.addressInternal = addressInternal;
+        this.connectorJarPaths = connectorJarPaths;
+        this.indexName =
+                INDEX_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+        HttpHost httpHost = HttpHost.create(addressExternal);
+        RestClientBuilder restClientBuilder = RestClient.builder(httpHost);
+        this.client = new RestHighLevelClient(restClientBuilder);
+    }
+
+    @Override
+    public Sink<TupleC2<String, String>> createSink(TestingSinkSettings sinkSettings) {
+        return new Elasticsearch6SinkBuilder<TupleC2<String, String>>()
+                .setHosts(HttpHost.create(this.addressInternal))
+                .setEmitter(new Elasticsearch6TestEmitter(indexName))
+                .setBulkFlushMaxActions(100) // emit after every element, don't buffer
+                .build();
+    }
+
+    @Override
+    public ExternalSystemDataReader<TupleC2<String, String>> createSinkDataReader(
+            TestingSinkSettings sinkSettings) {
+        Elasticsearch6Utils.refreshIndex(client, indexName);
+        return new Elasticsearch6DataReader(client, indexName, PAGE_LENGTH);
+    }
+
+    @Override
+    public List<TupleC2<String, String>> generateTestData(
+            TestingSinkSettings sinkSettings, long seed) {
+        Random random = new Random(seed);
+        List<TupleC2<String, String>> randomStringRecords = new ArrayList<>();
+        int recordNum =
+                random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND)
+                        + NUM_RECORDS_LOWER_BOUND;
+
+        for (int i = 0; i < recordNum; i++) {
+            int valueLength = random.nextInt(RANDOM_STRING_MAX_LENGTH) + 1;
+            String key = Integer.toString(i);
+            String value = RandomStringUtils.random(valueLength, true, true);
+            randomStringRecords.add(TupleC2.of(key, value));
+        }
+        return randomStringRecords;

Review Comment:
   Nit: Again, please use the stream syntax.



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkExternalContext.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.http.HttpHost;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+class Elasticsearch6SinkExternalContext
+        implements DataStreamSinkV2ExternalContext<TupleC2<String, String>> {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(Elasticsearch6SinkExternalContext.class);
+
+    private static final String INDEX_NAME_PREFIX = "es-index";
+    private static final int RANDOM_STRING_MAX_LENGTH = 50;
+    private static final int NUM_RECORDS_UPPER_BOUND = 500;
+    private static final int NUM_RECORDS_LOWER_BOUND = 100;
+    private static final int PAGE_LENGTH = 100;
+
+    protected final String indexName;
+
+    private final String addressInternal;
+    private final List<URL> connectorJarPaths;
+    private final RestHighLevelClient client;
+
+    Elasticsearch6SinkExternalContext(
+            String addressExternal, String addressInternal, List<URL> connectorJarPaths) {
+        this.addressInternal = addressInternal;
+        this.connectorJarPaths = connectorJarPaths;
+        this.indexName =
+                INDEX_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+        HttpHost httpHost = HttpHost.create(addressExternal);
+        RestClientBuilder restClientBuilder = RestClient.builder(httpHost);
+        this.client = new RestHighLevelClient(restClientBuilder);
+    }
+
+    @Override
+    public Sink<TupleC2<String, String>> createSink(TestingSinkSettings sinkSettings) {
+        return new Elasticsearch6SinkBuilder<TupleC2<String, String>>()
+                .setHosts(HttpHost.create(this.addressInternal))
+                .setEmitter(new Elasticsearch6TestEmitter(indexName))
+                .setBulkFlushMaxActions(100) // emit after every element, don't buffer
+                .build();
+    }
+
+    @Override
+    public ExternalSystemDataReader<TupleC2<String, String>> createSinkDataReader(
+            TestingSinkSettings sinkSettings) {
+        Elasticsearch6Utils.refreshIndex(client, indexName);
+        return new Elasticsearch6DataReader(client, indexName, PAGE_LENGTH);
+    }
+
+    @Override
+    public List<TupleC2<String, String>> generateTestData(
+            TestingSinkSettings sinkSettings, long seed) {
+        Random random = new Random(seed);
+        List<TupleC2<String, String>> randomStringRecords = new ArrayList<>();
+        int recordNum =
+                random.nextInt(NUM_RECORDS_UPPER_BOUND - NUM_RECORDS_LOWER_BOUND)
+                        + NUM_RECORDS_LOWER_BOUND;
+
+        for (int i = 0; i < recordNum; i++) {
+            int valueLength = random.nextInt(RANDOM_STRING_MAX_LENGTH) + 1;
+            String key = Integer.toString(i);
+            String value = RandomStringUtils.random(valueLength, true, true);
+            randomStringRecords.add(TupleC2.of(key, value));
+        }
+        return randomStringRecords;
+    }
+
+    @Override
+    public void close() {
+        DeleteIndexRequest request = new DeleteIndexRequest(indexName);
+        try {
+            client.indices().delete(request, RequestOptions.DEFAULT);
+        } catch (IOException e) {
+            LOG.error("Cannot delete index {}", indexName, e);
+        }
+        // This is needed to avoid race conditions between tests that reuse the same index
+        Elasticsearch6Utils.refreshIndex(client, indexName);

Review Comment:
   Out of curiosity since every time you instantiate the DataReader the indices are refreshed can you give some more details about the race condition?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch7/pom.xml:
##########
@@ -0,0 +1,158 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-end-to-end-tests</artifactId>
+		<version>1.16-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+
+	<artifactId>flink-end-to-end-tests-elasticsearch7</artifactId>
+	<name>Flink : E2E Tests : Elasticsearch 7 Java</name>
+	<packaging>jar</packaging>
+
+	<properties>
+		<elasticsearch.version>7.10.2</elasticsearch.version>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-elasticsearch7</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>1.16-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-end-to-end-tests-common</artifactId>
+			<version>1.16-SNAPSHOT</version>
+			<scope>test</scope>
+		</dependency>
+		<!--		<dependency>

Review Comment:
   Why does this module not depend on elasticsearch common? 



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkE2ECase.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestContext;
+import org.apache.flink.connector.testframe.junit.annotations.TestEnv;
+import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem;
+import org.apache.flink.connector.testframe.junit.annotations.TestSemantics;
+import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.flink.FlinkContainerTestEnvironment;
+import org.apache.flink.util.DockerImageVersions;
+
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.util.Arrays;
+
+/** End to end test for Elasticsearch7Sink based on connector testing framework. */
+@SuppressWarnings("unused")
+public class Elasticsearch6SinkE2ECase extends SinkTestSuiteBase<TupleC2<String, String>> {
+    private static final String ELASTICSEARCH_HOSTNAME = "elasticsearch";
+
+    @TestSemantics
+    CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE};
+
+    // Defines TestEnvironment
+    @TestEnv FlinkContainerTestEnvironment flink = new FlinkContainerTestEnvironment(1, 6);
+
+    // Defines ConnectorExternalSystem
+    @TestExternalSystem
+    DefaultContainerizedExternalSystem<ElasticsearchContainer> elasticsearch =
+            DefaultContainerizedExternalSystem.builder()
+                    .fromContainer(
+                            new ElasticsearchContainer(
+                                            DockerImageName.parse(
+                                                    DockerImageVersions.ELASTICSEARCH_7))
+                                    .withNetworkAliases(ELASTICSEARCH_HOSTNAME))
+                    .bindWithFlinkContainer(flink.getFlinkContainers().getJobManager())
+                    .build();
+
+    @TestContext
+    Elasticsearch6SinkExternalContextFactory contextFactory =
+            new Elasticsearch6SinkExternalContextFactory(
+                    elasticsearch.getContainer(),
+                    Arrays.asList(
+                            TestUtils.getResource("elasticsearch6-end-to-end-test.jar")
+                                    .toAbsolutePath()
+                                    .toUri()
+                                    .toURL(),
+                            TestUtils.getResource("flink-connector-testing-elasticsearch.jar")
+                                    .toAbsolutePath()
+                                    .toUri()
+                                    .toURL(),
+                            TestUtils.getResource("flink-connector-testing.jar")
+                                    .toAbsolutePath()
+                                    .toUri()

Review Comment:
   Can you explain why you need all three jars?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkExternalContextFactory.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalContextFactory;
+
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+
+import java.net.URL;
+import java.util.List;
+
+/** Elasticsearch sink external context factory. */
+public class Elasticsearch6SinkExternalContextFactory
+        implements ExternalContextFactory<Elasticsearch6SinkExternalContext> {
+
+    private final ElasticsearchContainer elasticsearchContainer;
+    private final List<URL> connectorJars;
+
+    Elasticsearch6SinkExternalContextFactory(
+            ElasticsearchContainer elasticsearchContainer, List<URL> connectorJars) {
+        this.elasticsearchContainer = elasticsearchContainer;
+        this.connectorJars = connectorJars;

Review Comment:
   `checkNotNull`



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6DataReader.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/** Elasticsearch data reader. */
+public class Elasticsearch6DataReader implements ExternalSystemDataReader<TupleC2<String, String>> {
+    private final RestHighLevelClient client;
+    private final String indexName;
+    private final int pageLength;
+    private int from = 0;
+
+    public Elasticsearch6DataReader(RestHighLevelClient client, String indexName, int pageLength) {
+        this.client = client;
+        this.indexName = indexName;
+        this.pageLength = pageLength;
+    }
+
+    @Override
+    public List<TupleC2<String, String>> poll(Duration timeout) {
+        List<TupleC2<String, String>> result = new ArrayList<>();
+        try {
+            SearchResponse response =
+                    client.search(
+                            new SearchRequest(indexName)
+                                    .source(
+                                            new SearchSourceBuilder()
+                                                    .query(QueryBuilders.matchAllQuery())
+                                                    .from(from)
+                                                    .size(pageLength)
+                                                    .trackTotalHits(true)),
+                            RequestOptions.DEFAULT);
+            SearchHit[] searchHits = response.getHits().getHits();
+            for (SearchHit searchHit : searchHits) {
+                TupleC2<String, String> hit =
+                        TupleC2.of(
+                                searchHit.getId(),
+                                searchHit.getSourceAsMap().get("data").toString());
+                result.add(hit);

Review Comment:
   Nit: I think you can replace this whole block with java streams API.
   ```java
   from += searchHits.length;
   return Arrays.stream(searchHits())
       .map(...)
       .collect(...);
   ```



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkExternalContext.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.http.HttpHost;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+class Elasticsearch6SinkExternalContext
+        implements DataStreamSinkV2ExternalContext<TupleC2<String, String>> {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(Elasticsearch6SinkExternalContext.class);
+
+    private static final String INDEX_NAME_PREFIX = "es-index";
+    private static final int RANDOM_STRING_MAX_LENGTH = 50;
+    private static final int NUM_RECORDS_UPPER_BOUND = 500;
+    private static final int NUM_RECORDS_LOWER_BOUND = 100;
+    private static final int PAGE_LENGTH = 100;
+
+    protected final String indexName;
+
+    private final String addressInternal;
+    private final List<URL> connectorJarPaths;
+    private final RestHighLevelClient client;
+
+    Elasticsearch6SinkExternalContext(
+            String addressExternal, String addressInternal, List<URL> connectorJarPaths) {
+        this.addressInternal = addressInternal;
+        this.connectorJarPaths = connectorJarPaths;

Review Comment:
   `checkNotNull`



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6SinkExternalContext.java:
##########
@@ -0,0 +1,135 @@
+/*
+ * 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 com.apache.flink.streaming.tests;
+
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder;
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext;
+import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.http.HttpHost;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+class Elasticsearch6SinkExternalContext
+        implements DataStreamSinkV2ExternalContext<TupleC2<String, String>> {
+
+    private static final Logger LOG =
+            LoggerFactory.getLogger(Elasticsearch6SinkExternalContext.class);
+
+    private static final String INDEX_NAME_PREFIX = "es-index";
+    private static final int RANDOM_STRING_MAX_LENGTH = 50;
+    private static final int NUM_RECORDS_UPPER_BOUND = 500;
+    private static final int NUM_RECORDS_LOWER_BOUND = 100;
+    private static final int PAGE_LENGTH = 100;
+
+    protected final String indexName;
+
+    private final String addressInternal;
+    private final List<URL> connectorJarPaths;
+    private final RestHighLevelClient client;
+
+    Elasticsearch6SinkExternalContext(
+            String addressExternal, String addressInternal, List<URL> connectorJarPaths) {
+        this.addressInternal = addressInternal;
+        this.connectorJarPaths = connectorJarPaths;
+        this.indexName =
+                INDEX_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE);
+        HttpHost httpHost = HttpHost.create(addressExternal);
+        RestClientBuilder restClientBuilder = RestClient.builder(httpHost);
+        this.client = new RestHighLevelClient(restClientBuilder);
+    }
+
+    @Override
+    public Sink<TupleC2<String, String>> createSink(TestingSinkSettings sinkSettings) {
+        return new Elasticsearch6SinkBuilder<TupleC2<String, String>>()
+                .setHosts(HttpHost.create(this.addressInternal))
+                .setEmitter(new Elasticsearch6TestEmitter(indexName))
+                .setBulkFlushMaxActions(100) // emit after every element, don't buffer
+                .build();
+    }
+
+    @Override
+    public ExternalSystemDataReader<TupleC2<String, String>> createSinkDataReader(
+            TestingSinkSettings sinkSettings) {
+        Elasticsearch6Utils.refreshIndex(client, indexName);

Review Comment:
   Would it make sense to move the refresh method into `Elasticsearch6DataReader#poll` ?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/test/java/com/apache/flink/streaming/tests/Elasticsearch6Utils.java:
##########
@@ -0,0 +1,43 @@
+package com.apache.flink.streaming.tests;
+
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.admin.indices.flush.FlushRequest;
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.rest.RestStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+class Elasticsearch6Utils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6Utils.class);
+
+    static void refreshIndex(RestHighLevelClient client, String indexName) {
+        RefreshRequest refresh = new RefreshRequest(indexName);
+        try {
+            client.indices().refresh(refresh, RequestOptions.DEFAULT);
+        } catch (IOException e) {
+            LOG.error("Cannot delete index {}", indexName);
+        } catch (ElasticsearchException e) {
+            if (e.status() == RestStatus.NOT_FOUND) {
+                LOG.info("Index {} not found", indexName);
+            }
+        }
+    }
+
+    static void flushIndex(RestHighLevelClient client, String indexName) {

Review Comment:
   This method seems to be unused.



-- 
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: issues-unsubscribe@flink.apache.org

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