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


##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch-common/src/main/java/org/apache/flink/streaming/tests/KeyValue.java:
##########
@@ -0,0 +1,74 @@
+package org.apache.flink.streaming.tests;
+
+import org.apache.flink.util.StringUtils;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/** A {@link Comparable} holder for key-value pairs. */
+public class KeyValue<K extends Comparable<? super K>, V extends Comparable<? 
super V>>
+        implements Comparable<KeyValue<K, V>>, Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    /** The key of the key-value pair. */
+    public K key;
+    /** The value the key-value pair. */
+    public V value;
+
+    /** Creates a new key-value pair where all fields are null. */
+    public KeyValue() {}
+
+    private KeyValue(K key, V value) {
+        this.key = key;
+        this.value = value;
+    }
+
+    @Override
+    public int compareTo(KeyValue<K, V> other) {

Review Comment:
   Do you need the `compareTo`? If you only need the comparison in a single 
place, it might be easier to use the existing `Tuple2` class and just pass in 
the respective comparator at the place where you want to compare things.



##########
flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java:
##########
@@ -508,6 +509,7 @@ private void checkResultWithSemantic(
                                 
.matchesRecordsFromSource(Arrays.asList(sort(testData)), semantic);
                         return true;
                     } catch (Throwable t) {
+                        LOG.error("Ooops", t);

Review Comment:
   Please use a more descriptive error message.



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/pom.xml:
##########
@@ -0,0 +1,127 @@
+<?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-end-to-end-tests-elasticsearch-common</artifactId>
+                       <version>${project.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-end-to-end-tests-common</artifactId>
+                       <version>${project.version}</version>
+                       <scope>test</scope>
+               </dependency>
+       </dependencies>
+
+       <build>
+               <plugins>
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-shade-plugin</artifactId>
+                               <executions>
+                                       <execution>
+                                               <goals>
+                                                       <goal>shade</goal>

Review Comment:
   Quick reminder why do we need shading here?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch-common/src/main/java/org/apache/flink/streaming/tests/ElasticsearchDataReader.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.flink.streaming.tests;
+
+import org.apache.flink.connector.testframe.external.ExternalSystemDataReader;
+
+import java.time.Duration;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** Elasticsearch data reader. */
+public class ElasticsearchDataReader
+        implements ExternalSystemDataReader<KeyValue<Integer, String>> {
+    private final ElasticsearchClient client;
+    private final String indexName;
+    private final int pageLength;
+    private int from;
+
+    public ElasticsearchDataReader(ElasticsearchClient client, String 
indexName, int pageLength) {
+        this.client = checkNotNull(client);
+        this.indexName = checkNotNull(indexName);
+        this.pageLength = pageLength;
+    }
+
+    @Override
+    public List<KeyValue<Integer, String>> poll(Duration timeout) {
+        client.refreshIndex(indexName);
+        // TODO: Tests are flaky without this small delay.

Review Comment:
   I thought we had fixed this problem. Can you explain what is going on here?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch6/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6Client.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.flink.streaming.tests;
+
+import org.apache.http.HttpHost;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+import org.elasticsearch.search.sort.SortOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The type Elasticsearch 6 client. */
+public class Elasticsearch6Client implements ElasticsearchClient {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(Elasticsearch6Client.class);
+
+    private final RestHighLevelClient restClient;
+
+    /**
+     * Instantiates a new Elasticsearch 6 client.
+     *
+     * @param addressExternal The address to access Elasticsearch from the 
host machine (outside of
+     *     the containerized environment).
+     */
+    public Elasticsearch6Client(String addressExternal) {
+        checkNotNull(addressExternal);
+        HttpHost httpHost = HttpHost.create(addressExternal);
+        RestClientBuilder restClientBuilder = RestClient.builder(httpHost);
+        this.restClient = new RestHighLevelClient(restClientBuilder);
+        checkNotNull(restClient);
+    }
+
+    @Override
+    public void deleteIndex(String indexName) {
+        DeleteIndexRequest request = new DeleteIndexRequest(indexName);
+        try {
+            restClient.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
+        refreshIndex(indexName);
+    }
+
+    @Override
+    public void refreshIndex(String indexName) {
+        RefreshRequest refresh = new RefreshRequest(indexName);
+        try {
+            restClient.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);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        restClient.close();
+    }
+
+    @Override
+    public List<KeyValue<Integer, String>> fetchAll(QueryParams params) {
+        try {
+            SearchResponse response =
+                    restClient.search(
+                            new SearchRequest(params.indexName())
+                                    .source(
+                                            new SearchSourceBuilder()
+                                                    .sort(params.sortField(), 
SortOrder.ASC)
+                                                    .from(params.from())
+                                                    .size(params.pageLength())
+                                                    
.trackTotalHits(params.trackTotalHits())),
+                            RequestOptions.DEFAULT);
+            SearchHit[] searchHits = response.getHits().getHits();
+            return Arrays.stream(searchHits)
+                    .map(
+                            searchHit ->
+                                    KeyValue.of(
+                                            Integer.valueOf(searchHit.getId()),
+                                            
searchHit.getSourceAsMap().get("value").toString()))
+                    .collect(Collectors.toList());
+        } catch (IOException e) {
+            return Collections.emptyList();

Review Comment:
   Add a log statement to increase the awareness of the error.



##########
flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java:
##########
@@ -500,6 +500,7 @@ private void checkResultWithSemantic(
             ExternalSystemDataReader<T> reader, List<T> testData, 
CheckpointingMode semantic)
             throws Exception {
         final ArrayList<T> result = new ArrayList<>();
+        int failed = 0;

Review Comment:
   Is this variable unused?



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch7/pom.xml:
##########
@@ -0,0 +1,128 @@
+<?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-end-to-end-tests-elasticsearch-common</artifactId>
+                       <version>${project.version}</version>
+               </dependency>
+
+               <dependency>
+                       <groupId>org.apache.flink</groupId>
+                       <artifactId>flink-end-to-end-tests-common</artifactId>
+                       <version>${project.version}</version>
+                       <scope>test</scope>
+               </dependency>
+       </dependencies>
+
+       <build>
+               <plugins>
+                       <plugin>
+                               <groupId>org.apache.maven.plugins</groupId>
+                               <artifactId>maven-shade-plugin</artifactId>
+                               <executions>
+                                       <execution>
+                                               <phase>package</phase>
+                                               <goals>
+                                                       <goal>shade</goal>

Review Comment:
   Same question about the shading



##########
flink-end-to-end-tests/flink-end-to-end-tests-elasticsearch7/src/main/java/org/apache/flink/streaming/tests/Elasticsearch7Client.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.flink.streaming.tests;
+
+import org.apache.http.HttpHost;
+import org.elasticsearch.ElasticsearchException;
+import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestClientBuilder;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+import org.elasticsearch.search.sort.SortOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The type Elasticsearch 7 client. */
+public class Elasticsearch7Client implements ElasticsearchClient {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(Elasticsearch7Client.class);
+
+    private final RestHighLevelClient restClient;
+
+    /**
+     * Instantiates a new Elasticsearch 7 client.
+     *
+     * @param addressExternal The address to access Elasticsearch from the 
host machine (outside of
+     *     the containerized environment).
+     */
+    public Elasticsearch7Client(String addressExternal) {
+        checkNotNull(addressExternal);
+        HttpHost httpHost = HttpHost.create(addressExternal);
+        RestClientBuilder restClientBuilder = RestClient.builder(httpHost);
+        this.restClient = new RestHighLevelClient(restClientBuilder);
+        checkNotNull(restClient);
+    }
+
+    @Override
+    public void deleteIndex(String indexName) {
+        DeleteIndexRequest request = new DeleteIndexRequest(indexName);
+        try {
+            restClient.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
+        refreshIndex(indexName);
+    }
+
+    @Override
+    public void refreshIndex(String indexName) {
+        RefreshRequest refresh = new RefreshRequest(indexName);
+        try {
+            restClient.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);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws Exception {
+        restClient.close();
+    }
+
+    @Override
+    public List<KeyValue<Integer, String>> fetchAll(QueryParams params) {
+        try {
+            SearchResponse response =
+                    restClient.search(
+                            new SearchRequest(params.indexName())
+                                    .source(
+                                            new SearchSourceBuilder()
+                                                    .sort(params.sortField(), 
SortOrder.ASC)
+                                                    .from(params.from())
+                                                    .size(params.pageLength())
+                                                    
.trackTotalHits(params.trackTotalHits())),
+                            RequestOptions.DEFAULT);
+            SearchHit[] searchHits = response.getHits().getHits();
+            return Arrays.stream(searchHits)
+                    .map(
+                            searchHit ->
+                                    KeyValue.of(
+                                            Integer.valueOf(searchHit.getId()),
+                                            
searchHit.getSourceAsMap().get("value").toString()))
+                    .collect(Collectors.toList());
+        } catch (IOException e) {
+            return Collections.emptyList();

Review Comment:
   Again add log



##########
flink-end-to-end-tests/run-nightly-tests.sh:
##########
@@ -199,9 +199,6 @@ function run_group_2 {
 
     run_test "Netty shuffle direct memory consumption end-to-end test" 
"$END_TO_END_DIR/test-scripts/test_netty_shuffle_memory_control.sh"
 
-    run_test "Elasticsearch (v6.8.20) sink end-to-end test" 
"$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 
https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.8.20.tar.gz";

Review Comment:
   It would be great to remove the bash e2e tests on a separate commit.



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

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

Reply via email to