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


##########
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:
   Copy-paste from Kafka tests. Migrated.



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