From b0a4fcf6b8af0e759c35aa605052ccc77bae7abe Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 12 Jan 2017 15:21:56 +0200 Subject: [PATCH 001/207] [FLINK-4988] [elasticsearch] Restructure Elasticsearch connectors This closes #3112. --- .../pom.xml | 95 +++++++ .../elasticsearch/BulkProcessorIndexer.java | 44 ++++ .../ElasticsearchApiCallBridge.java | 60 +++++ .../elasticsearch/ElasticsearchSinkBase.java | 237 ++++++++++++++++++ .../ElasticsearchSinkFunction.java | 71 ++++++ .../elasticsearch/RequestIndexer.java | 37 +++ .../util/ElasticsearchUtils.java | 51 ++++ .../ElasticsearchSinkTestBase.java | 186 ++++++++++++++ .../EmbeddedElasticsearchNodeEnvironment.java | 55 ++++ .../testutils/SourceSinkDataTestKit.java | 112 +++++++++ .../src/test/resources/log4j-test.properties | 27 ++ .../src/test/resources/logback-test.xml | 30 +++ 12 files changed, 1005 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/pom.xml create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml new file mode 100644 index 00000000..81652c45 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -0,0 +1,95 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.3-SNAPSHOT + .. + + + flink-connector-elasticsearch-base_2.10 + flink-connector-elasticsearch-base + + jar + + + + 1.7.1 + + + + + + + + org.apache.flink + flink-streaming-java_2.10 + ${project.version} + provided + + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + + + + + + org.apache.flink + flink-test-utils_2.10 + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_2.10 + ${project.version} + test + test-jar + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java new file mode 100644 index 00000000..d8025501 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -0,0 +1,44 @@ +/* + * 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.connectors.elasticsearch; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. + * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + */ +class BulkProcessorIndexer implements RequestIndexer { + + private static final long serialVersionUID = 6841162943062034253L; + + private final BulkProcessor bulkProcessor; + + BulkProcessorIndexer(BulkProcessor bulkProcessor) { + this.bulkProcessor = bulkProcessor; + } + + @Override + public void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + this.bulkProcessor.add(actionRequest); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java new file mode 100644 index 00000000..6298a857 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -0,0 +1,60 @@ +/* + * 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.connectors.elasticsearch; + +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.client.Client; + +import javax.annotation.Nullable; +import java.io.Serializable; +import java.util.Map; + +/** + * An {@link ElasticsearchApiCallBridge} is used to bridge incompatible Elasticsearch Java API calls across different versions. + * This includes calls to create Elasticsearch clients, handle failed item responses, etc. Any incompatible Elasticsearch + * Java APIs should be bridged using this interface. + * + * Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node + * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold + * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. + */ +public interface ElasticsearchApiCallBridge extends Serializable { + + /** + * Creates an Elasticsearch {@link Client}. + * + * @param clientConfig The configuration to use when constructing the client. + * @return The created client. + */ + Client createClient(Map clientConfig); + + /** + * Extracts the cause of failure of a bulk item action. + * + * @param bulkItemResponse the bulk item response to extract cause of failure + * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). + */ + @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + + /** + * Perform any necessary state cleanup. + */ + void cleanup(); + +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java new file mode 100644 index 00000000..6a2d65fe --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -0,0 +1,237 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; +import org.apache.flink.util.InstantiationUtil; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Base class for all Flink Elasticsearch Sinks. + * + *

+ * This class implements the common behaviour across Elasticsearch versions, such as + * the use of an internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before + * sending the requests to the cluster, as well as passing input records to the user provided + * {@link ElasticsearchSinkFunction} for processing. + * + *

+ * The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of + * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, + * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. + * + * @param Type of the elements handled by this sink + */ +public abstract class ElasticsearchSinkBase extends RichSinkFunction { + + private static final long serialVersionUID = -1007596293618451942L; + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkBase.class); + + // ------------------------------------------------------------------------ + // Internal bulk processor configuration + // ------------------------------------------------------------------------ + + public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; + public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; + public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + + private final Integer bulkProcessorFlushMaxActions; + private final Integer bulkProcessorFlushMaxSizeMb; + private final Integer bulkProcessorFlushIntervalMillis; + + // ------------------------------------------------------------------------ + // User-facing API and configuration + // ------------------------------------------------------------------------ + + /** The user specified config map that we forward to Elasticsearch when we create the {@link Client}. */ + private final Map userConfig; + + /** The function that is used to construct mulitple {@link ActionRequest ActionRequests} from each incoming element. */ + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ + private transient BulkProcessorIndexer requestIndexer; + + // ------------------------------------------------------------------------ + // Internals for the Flink Elasticsearch Sink + // ------------------------------------------------------------------------ + + /** Call bridge for different version-specfic */ + private final ElasticsearchApiCallBridge callBridge; + + /** Elasticsearch client created using the call bridge. */ + private transient Client client; + + /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ + private transient BulkProcessor bulkProcessor; + + /** + * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown in callbacks. + */ + private final AtomicReference failureThrowable = new AtomicReference<>(); + + public ElasticsearchSinkBase( + ElasticsearchApiCallBridge callBridge, + Map userConfig, + ElasticsearchSinkFunction elasticsearchSinkFunction) { + + this.callBridge = checkNotNull(callBridge); + this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); + + // we eagerly check if the user-provided sink function is serializable; + // otherwise, if it isn't serializable, users will merely get a non-informative error message + // "ElasticsearchSinkBase is not serializable" + try { + InstantiationUtil.serializeObject(elasticsearchSinkFunction); + } catch (Exception e) { + throw new IllegalArgumentException( + "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + + "The object probably contains or references non serializable fields."); + } + + checkNotNull(userConfig); + + // extract and remove bulk processor related configuration from the user-provided config, + // so that the resulting user config only contains configuration related to the Elasticsearch client. + ParameterTool params = ParameterTool.fromMap(userConfig); + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { + bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + } else { + bulkProcessorFlushMaxActions = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { + bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + } else { + bulkProcessorFlushMaxSizeMb = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { + bulkProcessorFlushIntervalMillis = params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + } else { + bulkProcessorFlushIntervalMillis = null; + } + + this.userConfig = userConfig; + } + + @Override + public void open(Configuration parameters) throws Exception { + client = callBridge.createClient(userConfig); + + BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder( + client, + new BulkProcessor.Listener() { + @Override + public void beforeBulk(long executionId, BulkRequest request) { } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + if (response.hasFailures()) { + for (BulkItemResponse itemResp : response.getItems()) { + Throwable failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResp); + if (failure != null) { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + failureThrowable.compareAndSet(null, failure); + } + } + } + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); + failureThrowable.compareAndSet(null, failure); + } + } + ); + + // This makes flush() blocking + bulkProcessorBuilder.setConcurrentRequests(0); + + if (bulkProcessorFlushMaxActions != null) { + bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions); + } + + if (bulkProcessorFlushMaxSizeMb != null) { + bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, ByteSizeUnit.MB)); + } + + if (bulkProcessorFlushIntervalMillis != null) { + bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + } + + bulkProcessor = bulkProcessorBuilder.build(); + requestIndexer = new BulkProcessorIndexer(bulkProcessor); + } + + @Override + public void invoke(T value) throws Exception { + // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink + checkErrorAndRethrow(); + + elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + } + + @Override + public void close() throws Exception { + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + client = null; + } + + callBridge.cleanup(); + + // make sure any errors from callbacks are rethrown + checkErrorAndRethrow(); + } + + private void checkErrorAndRethrow() { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occured in ElasticsearchSink.", cause); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java new file mode 100644 index 00000000..1e20a0a3 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -0,0 +1,71 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream. + * + *

+ * This is used by sinks to prepare elements for sending them to Elasticsearch. + * + *

+ * Example: + * + *

{@code
+ *					private static class TestElasticSearchSinkFunction implements
+ *						ElasticsearchSinkFunction> {
+ *
+ *					public IndexRequest createIndexRequest(Tuple2 element) {
+ *						Map json = new HashMap<>();
+ *						json.put("data", element.f1);
+ *
+ *						return Requests.indexRequest()
+ *							.index("my-index")
+ *							.type("my-type")
+ *							.id(element.f0.toString())
+ *							.source(json);
+ *						}
+ *
+ *				public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
+ *					indexer.add(createIndexRequest(element));
+ *				}
+ *		}
+ *
+ * }
+ * + * @param The type of the element handled by this {@code ElasticsearchSinkFunction} + */ +public interface ElasticsearchSinkFunction extends Serializable, Function { + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. + * The produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param ctx runtime context containing information about the sink instance + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void process(T element, RuntimeContext ctx, RequestIndexer indexer); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java new file mode 100644 index 00000000..4587a807 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -0,0 +1,37 @@ +/* + * 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.connectors.elasticsearch; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * Users add multiple {@link ActionRequest ActionRequests} to a {@link RequestIndexer} to prepare + * them for sending to an Elasticsearch cluster. + */ +public interface RequestIndexer extends Serializable { + + /** + * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param actionRequests The multiple {@link ActionRequest} to add. + */ + void add(ActionRequest... actionRequests); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java new file mode 100644 index 00000000..9776c4c9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.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 org.apache.flink.streaming.connectors.elasticsearch.util; + +import org.elasticsearch.common.transport.InetSocketTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; + +/** + * Suite of utility methods for Elasticsearch. + */ +public class ElasticsearchUtils { + + /** + * Utility method to convert a {@link List} of {@link InetSocketAddress} to Elasticsearch {@link TransportAddress}. + * + * @param inetSocketAddresses The list of {@link InetSocketAddress} to convert. + */ + public static List convertInetSocketAddresses(List inetSocketAddresses) { + if (inetSocketAddresses == null) { + return null; + } else { + List converted; + converted = new ArrayList<>(inetSocketAddresses.size()); + for (InetSocketAddress address : inetSocketAddresses) { + converted.add(new InetSocketTransportAddress(address)); + } + return converted; + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java new file mode 100644 index 00000000..2f9e4c17 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -0,0 +1,186 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; +import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.util.InstantiationUtil; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.transport.TransportClient; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} implementations. + */ +public abstract class ElasticsearchSinkTestBase extends StreamingMultipleProgramsTestBase { + + protected final static String CLUSTER_NAME = "test-cluster"; + + protected static EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; + + @ClassRule + public static TemporaryFolder tempFolder = new TemporaryFolder(); + + @BeforeClass + public static void prepare() throws Exception { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + // dynamically load version-specific implementation of the Elasticsearch embedded node environment + Class clazz = Class.forName( + "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); + embeddedNodeEnv = (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); + + embeddedNodeEnv.start(tempFolder.newFolder(), CLUSTER_NAME); + + } + + @AfterClass + public static void shutdown() throws Exception { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shutting down embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + embeddedNodeEnv.close(); + + } + + /** + * Tests that the Elasticsearch sink works properly using a {@link TransportClient}. + */ + public void runTransportClientTest() throws Exception { + final String index = "transport-client-test-index"; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put("cluster.name", CLUSTER_NAME); + + source.addSink(createElasticsearchSinkForEmbeddedNode( + userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + + env.execute("Elasticsearch TransportClient Test"); + + // verify the results + Client client = embeddedNodeEnv.getClient(); + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. + */ + public void runNullTransportClientTest() throws Exception { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put("cluster.name", "my-transport-client-cluster"); + + try { + createElasticsearchSink(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch(IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. + */ + public void runEmptyTransportClientTest() throws Exception { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put("cluster.name", "my-transport-client-cluster"); + + try { + createElasticsearchSink( + userConfig, + Collections.emptyList(), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch(IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. + */ + public void runTransportClientFailsTest() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put("cluster.name", "my-transport-client-cluster"); + + source.addSink(createElasticsearchSinkForEmbeddedNode( + userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + + try { + env.execute("Elasticsearch Transport Client Test"); + } catch(JobExecutionException expectedException) { + assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); + return; + } + + fail(); + } + + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses */ + protected abstract ElasticsearchSinkBase createElasticsearchSink(Map userConfig, + List transportAddresses, + ElasticsearchSinkFunction elasticsearchSinkFunction); + + /** + * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch node. + * + * This case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(Map, List, ElasticsearchSinkFunction)} + * because the Elasticsearch Java API to do so is incompatible across different versions. + */ + protected abstract ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( + Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception; +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java new file mode 100644 index 00000000..f59eb03c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -0,0 +1,55 @@ +/* + * 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.connectors.elasticsearch; + +import org.elasticsearch.client.Client; + +import java.io.File; + +/** + * The {@link EmbeddedElasticsearchNodeEnvironment} is used in integration tests to manage Elasticsearch embedded nodes. + * + * NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific implementations + * for the tests, concrete implementations must be named {@code EmbeddedElasticsearchNodeEnvironmentImpl}. It must + * also be located under the same package. The intentional package-private accessibility of this interface + * enforces that. + */ +interface EmbeddedElasticsearchNodeEnvironment { + + /** + * Start an embedded Elasticsearch node instance. + * Calling this method multiple times consecutively should not restart the embedded node. + * + * @param tmpDataFolder The temporary data folder for the embedded node to use. + * @param clusterName The name of the cluster that the embedded node should be configured with. + */ + void start(File tmpDataFolder, String clusterName) throws Exception; + + /** + * Close the embedded node, if previously started. + */ + void close() throws Exception; + + /** + * Returns a client to communicate with the embedded node. + * + * @return Client to communicate with the embedded node. + * Returns {@code null} if the embedded node wasn't started or is closed. + */ + Client getClient(); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java new file mode 100644 index 00000000..55a48fae --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -0,0 +1,112 @@ +/* + * 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.connectors.elasticsearch.testutils; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; +import org.junit.Assert; + +import java.util.HashMap; +import java.util.Map; + +/** + * This class contains utilities and a pre-defined source function and + * Elasticsearch Sink function used to simulate and verify data used in tests. + */ +public class SourceSinkDataTestKit { + + private static final int NUM_ELEMENTS = 20; + + private static final String DATA_PREFIX = "message #"; + private static final String DATA_FIELD_NAME = "data"; + + private static final String TYPE_NAME = "flink-es-test-type"; + + /** + * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 - 20. + */ + public static class TestDataSourceFunction implements SourceFunction> { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceFunction.SourceContext> ctx) throws Exception { + for (int i = 0; i < NUM_ELEMENTS && running; i++) { + ctx.collect(Tuple2.of(i, DATA_PREFIX + i)); + } + } + + @Override + public void cancel() { + running = false; + } + } + + /** + * A {@link ElasticsearchSinkFunction} that indexes each element it receives to a sepecified Elasticsearch index. + */ + public static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { + private static final long serialVersionUID = 1L; + + private final String index; + + /** + * Create the sink function, specifying a target Elasticsearch index. + * + * @param index Name of the target Elasticsearch index. + */ + public TestElasticsearchSinkFunction(String index) { + this.index = index; + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map json = new HashMap<>(); + json.put(DATA_FIELD_NAME, element.f1); + + return new IndexRequest(index, TYPE_NAME, element.f0.toString()).source(json); + } + + @Override + public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + } + + /** + * Verify the results in an Elasticsearch index. The results must first be produced into the index + * using a {@link TestElasticsearchSinkFunction}; + * + * @param client The client to use to connect to Elasticsearch + * @param index The index to check + */ + public static void verifyProducedSinkData(Client client, String index) { + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = client.get(new GetRequest(index, TYPE_NAME, Integer.toString(i))).actionGet(); + Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties new file mode 100644 index 00000000..20551848 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml new file mode 100644 index 00000000..45b3b92f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml @@ -0,0 +1,30 @@ + + + + + + %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n + + + + + + + + \ No newline at end of file From db8bf2c0c6fba8389bf0926d0c1d3c898dd6e900 Mon Sep 17 00:00:00 2001 From: Max Kuklinski Date: Wed, 23 Nov 2016 17:54:11 +0100 Subject: [PATCH 002/207] [FLINK-5122] [elasticsearch] Retry temporary Elasticsearch request errors. Covered exceptions are: Timeouts, No Master, UnavailableShardsException, bulk queue on node full --- .../elasticsearch/ElasticsearchSinkBase.java | 62 +++++++++++++++++-- 1 file changed, 58 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 6a2d65fe..7977fc09 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -21,12 +21,15 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.util.InstantiationUtil; +import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -85,6 +88,13 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction { /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient BulkProcessorIndexer requestIndexer; + /** + * When set to true and the bulk action fails, the error message will be checked for + * common patterns like timeout, UnavailableShardsException or a full buffer queue on the node. + * When a matching pattern is found, the bulk will be retried. + */ + protected boolean checkErrorAndRetryBulk = false; + // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink // ------------------------------------------------------------------------ @@ -165,20 +175,49 @@ public void beforeBulk(long executionId, BulkRequest request) { } @Override public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { if (response.hasFailures()) { + boolean allRequestsRepeatable = true; + for (BulkItemResponse itemResp : response.getItems()) { Throwable failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResp); if (failure != null) { - LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); - failureThrowable.compareAndSet(null, failure); + String failureMessageLowercase = itemResp.getFailureMessage().toLowerCase(); + + // Check if index request can be retried + if (checkErrorAndRetryBulk && ( + failureMessageLowercase.contains("timeout") || + failureMessageLowercase.contains("timed out") || // Generic timeout errors + failureMessageLowercase.contains("UnavailableShardsException".toLowerCase()) || // Shard not available due to rebalancing or node down + (failureMessageLowercase.contains("data/write/bulk") && failureMessageLowercase.contains("bulk")))) // Bulk index queue on node full + { + LOG.debug("Retry bulk: {}", itemResp.getFailureMessage()); + } else { + // Cannot retry action + allRequestsRepeatable = false; + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + failureThrowable.compareAndSet(null, failure); + } } } + + if (allRequestsRepeatable) { + reAddBulkRequest(request); + } } } @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); - failureThrowable.compareAndSet(null, failure); + if (checkErrorAndRetryBulk && ( + failure instanceof ClusterBlockException // Examples: "no master" + || failure instanceof ElasticsearchTimeoutException) // ElasticsearchTimeoutException sounded good, not seen in stress tests yet + ) + { + LOG.debug("Retry bulk on throwable: {}", failure.getMessage()); + reAddBulkRequest(request); + } else { + LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); + failureThrowable.compareAndSet(null, failure); + } } } ); @@ -228,6 +267,21 @@ public void close() throws Exception { checkErrorAndRethrow(); } + /** + * Adds all requests of the bulk to the BulkProcessor. Used when trying again. + * @param bulkRequest + */ + public void reAddBulkRequest(BulkRequest bulkRequest) { + //TODO Check what happens when bulk contains a DeleteAction and IndexActions and the DeleteAction fails because the document already has been deleted. This may not happen in typical Flink jobs. + + for (IndicesRequest req : bulkRequest.subRequests()) { + if (req instanceof ActionRequest) { + // There is no waiting time between index requests, so this may produce additional pressure on cluster + bulkProcessor.add((ActionRequest) req); + } + } + } + private void checkErrorAndRethrow() { Throwable cause = failureThrowable.get(); if (cause != null) { From 21f2695e4f8e7d6f45db2fb3b576de066f7149c3 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 30 Jan 2017 13:55:26 +0800 Subject: [PATCH 003/207] [FLINK-5353] [elasticsearch] User-provided failure handler for ElasticsearchSink This commit fixes both FLINK-5353 and FLINK-5122. It allows users to implement a failure handler to control how failed action requests are dealt with. The commit also includes general improvements to FLINK-5122: 1. Use the built-in backoff functionality in the Elasticsearch BulkProcessor (not available for Elasticsearch 1.x) 2. Migrate the `checkErrorAndRetryBulk` functionality to the new failure handler --- .../ActionRequestFailureHandler.java | 72 ++++++++ .../ElasticsearchApiCallBridge.java | 12 ++ .../elasticsearch/ElasticsearchSinkBase.java | 174 ++++++++++++------ .../util/NoOpActionRequestFailureHandler.java | 37 ++++ 4 files changed, 234 insertions(+), 61 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java new file mode 100644 index 00000000..45d04fcb --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -0,0 +1,72 @@ +/* + * 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.connectors.elasticsearch; + +import org.elasticsearch.action.ActionRequest; + +import java.io.Serializable; + +/** + * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how failed + * {@link ActionRequest ActionRequests} should be handled, ex. dropping them, reprocessing malformed documents, or + * simply requesting them to be sent to Elasticsearch again if the failure is only temporary. + * + *

+ * Example: + * + *

{@code
+ *
+ *	private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
+ *
+ *		@Override
+ *		boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer) {
+ *			// this example uses Apache Commons to search for nested exceptions
+ *
+ *			if (ExceptionUtils.indexOfThrowable(failure, EsRejectedExecutionException.class) >= 0) {
+ *				// full queue; re-add document for indexing
+ *				indexer.add(action);
+ *				return false;
+ *			} else if (ExceptionUtils.indexOfThrowable(failure, ElasticsearchParseException.class) {
+ *				// malformed document; simply drop request without failing sink
+ *				return false;
+ *			} else {
+ *				// for all other failures, fail the sink
+ *				return true;
+ *			}
+ *		}
+ *	}
+ *
+ * }
+ * + *

+ * The above example will let the sink re-add requests that failed due to queue capacity saturation and drop requests + * with malformed documents, without failing the sink. For all other failures, the sink will fail. + */ +public interface ActionRequestFailureHandler extends Serializable { + + /** + * Handle a failed {@link ActionRequest}. + * + * @param action the {@link ActionRequest} that failed due to the failure + * @param failure the cause of failure + * @param indexer request indexer to re-add the failed action, if intended to do so + * @return the implementation should return {@code true} if the sink should fail due to this failure, and {@code false} otherwise + */ + boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer); + +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 6298a857..b4824328 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.Client; import javax.annotation.Nullable; @@ -52,6 +53,17 @@ public interface ElasticsearchApiCallBridge extends Serializable { */ @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + /** + * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. + * The builder will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). + */ + void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); + /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 7977fc09..2c29865f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -21,24 +21,23 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.util.InstantiationUtil; -import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.Serializable; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -70,10 +69,56 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction { public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; + + public enum FlushBackoffType { + CONSTANT, + EXPONENTIAL + } + + public class BulkFlushBackoffPolicy implements Serializable { + + private static final long serialVersionUID = -6022851996101826049L; + + // the default values follow the Elasticsearch default settings for BulkProcessor + private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL; + private int maxRetryCount = 8; + private long delayMillis = 50; + + public FlushBackoffType getBackoffType() { + return backoffType; + } + + public int getMaxRetryCount() { + return maxRetryCount; + } + + public long getDelayMillis() { + return delayMillis; + } + + public void setBackoffType(FlushBackoffType backoffType) { + this.backoffType = checkNotNull(backoffType); + } + + public void setMaxRetryCount(int maxRetryCount) { + checkArgument(maxRetryCount > 0); + this.maxRetryCount = maxRetryCount; + } + + public void setDelayMillis(long delayMillis) { + checkArgument(delayMillis > 0); + this.delayMillis = delayMillis; + } + } private final Integer bulkProcessorFlushMaxActions; private final Integer bulkProcessorFlushMaxSizeMb; private final Integer bulkProcessorFlushIntervalMillis; + private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; // ------------------------------------------------------------------------ // User-facing API and configuration @@ -85,16 +130,12 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction { /** The function that is used to construct mulitple {@link ActionRequest ActionRequests} from each incoming element. */ private final ElasticsearchSinkFunction elasticsearchSinkFunction; + /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ + private final ActionRequestFailureHandler failureHandler; + /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient BulkProcessorIndexer requestIndexer; - /** - * When set to true and the bulk action fails, the error message will be checked for - * common patterns like timeout, UnavailableShardsException or a full buffer queue on the node. - * When a matching pattern is found, the bulk will be retried. - */ - protected boolean checkErrorAndRetryBulk = false; - // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink // ------------------------------------------------------------------------ @@ -109,21 +150,28 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction { private transient BulkProcessor bulkProcessor; /** - * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown in callbacks. + * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown in callbacks and + * the user considered it should fail the sink via the + * {@link ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, RequestIndexer)} method. + * + * Errors will be checked and rethrown before processing each input element, and when the sink is closed. */ private final AtomicReference failureThrowable = new AtomicReference<>(); public ElasticsearchSinkBase( ElasticsearchApiCallBridge callBridge, Map userConfig, - ElasticsearchSinkFunction elasticsearchSinkFunction) { + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler) { this.callBridge = checkNotNull(callBridge); this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); + this.failureHandler = checkNotNull(failureHandler); - // we eagerly check if the user-provided sink function is serializable; - // otherwise, if it isn't serializable, users will merely get a non-informative error message + // we eagerly check if the user-provided sink function and failure handler is serializable; + // otherwise, if they aren't serializable, users will merely get a non-informative error message // "ElasticsearchSinkBase is not serializable" + try { InstantiationUtil.serializeObject(elasticsearchSinkFunction); } catch (Exception e) { @@ -132,10 +180,19 @@ public ElasticsearchSinkBase( "The object probably contains or references non serializable fields."); } - checkNotNull(userConfig); + try { + InstantiationUtil.serializeObject(failureHandler); + } catch (Exception e) { + throw new IllegalArgumentException( + "The implementation of the provided ActionRequestFailureHandler is not serializable. " + + "The object probably contains or references non serializable fields."); + } // extract and remove bulk processor related configuration from the user-provided config, // so that the resulting user config only contains configuration related to the Elasticsearch client. + + checkNotNull(userConfig); + ParameterTool params = ParameterTool.fromMap(userConfig); if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { @@ -159,6 +216,31 @@ public ElasticsearchSinkBase( bulkProcessorFlushIntervalMillis = null; } + boolean bulkProcessorFlushBackoffEnable = params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE); + + if (bulkProcessorFlushBackoffEnable) { + this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy(); + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) { + bulkProcessorFlushBackoffPolicy.setBackoffType(FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE))); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) { + bulkProcessorFlushBackoffPolicy.setMaxRetryCount(params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) { + bulkProcessorFlushBackoffPolicy.setDelayMillis(params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY); + } + + } else { + bulkProcessorFlushBackoffPolicy = null; + } + this.userConfig = userConfig; } @@ -175,48 +257,30 @@ public void beforeBulk(long executionId, BulkRequest request) { } @Override public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { if (response.hasFailures()) { - boolean allRequestsRepeatable = true; + BulkItemResponse itemResponse; + Throwable failure; - for (BulkItemResponse itemResp : response.getItems()) { - Throwable failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResp); + for (int i = 0; i < response.getItems().length; i++) { + itemResponse = response.getItems()[i]; + failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); if (failure != null) { - String failureMessageLowercase = itemResp.getFailureMessage().toLowerCase(); - - // Check if index request can be retried - if (checkErrorAndRetryBulk && ( - failureMessageLowercase.contains("timeout") || - failureMessageLowercase.contains("timed out") || // Generic timeout errors - failureMessageLowercase.contains("UnavailableShardsException".toLowerCase()) || // Shard not available due to rebalancing or node down - (failureMessageLowercase.contains("data/write/bulk") && failureMessageLowercase.contains("bulk")))) // Bulk index queue on node full - { - LOG.debug("Retry bulk: {}", itemResp.getFailureMessage()); - } else { - // Cannot retry action - allRequestsRepeatable = false; - LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); + if (failureHandler.onFailure(request.requests().get(i), failure, requestIndexer)) { failureThrowable.compareAndSet(null, failure); } } } - - if (allRequestsRepeatable) { - reAddBulkRequest(request); - } } } @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - if (checkErrorAndRetryBulk && ( - failure instanceof ClusterBlockException // Examples: "no master" - || failure instanceof ElasticsearchTimeoutException) // ElasticsearchTimeoutException sounded good, not seen in stress tests yet - ) - { - LOG.debug("Retry bulk on throwable: {}", failure.getMessage()); - reAddBulkRequest(request); - } else { - LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); - failureThrowable.compareAndSet(null, failure); + LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure.getCause()); + + // whole bulk request failures are usually just temporary timeouts on + // the Elasticsearch side; simply retry all action requests in the bulk + for (ActionRequest action : request.requests()) { + requestIndexer.add(action); } } } @@ -237,6 +301,9 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); } + // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null + callBridge.configureBulkProcessorBackoff(bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); + bulkProcessor = bulkProcessorBuilder.build(); requestIndexer = new BulkProcessorIndexer(bulkProcessor); } @@ -267,21 +334,6 @@ public void close() throws Exception { checkErrorAndRethrow(); } - /** - * Adds all requests of the bulk to the BulkProcessor. Used when trying again. - * @param bulkRequest - */ - public void reAddBulkRequest(BulkRequest bulkRequest) { - //TODO Check what happens when bulk contains a DeleteAction and IndexActions and the DeleteAction fails because the document already has been deleted. This may not happen in typical Flink jobs. - - for (IndicesRequest req : bulkRequest.subRequests()) { - if (req instanceof ActionRequest) { - // There is no waiting time between index requests, so this may produce additional pressure on cluster - bulkProcessor.add((ActionRequest) req); - } - } - } - private void checkErrorAndRethrow() { Throwable cause = failureThrowable.get(); if (cause != null) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java new file mode 100644 index 00000000..09173a2f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java @@ -0,0 +1,37 @@ +/* + * 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.connectors.elasticsearch.util; + +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.elasticsearch.action.ActionRequest; + +/** + * An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. + */ +public class NoOpActionRequestFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 737941343410827885L; + + @Override + public boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer) { + // simply fail the sink + return true; + } + +} From 2da29bcaa1a22477b6c3f0db6b693c6df6ec4c41 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 20 Feb 2017 16:50:19 +0800 Subject: [PATCH 004/207] [FLINK-5487] [elasticsearch] At-least-once Elasticsearch Sink This closes #3358. --- .../pom.xml | 8 + .../ActionRequestFailureHandler.java | 29 +- .../elasticsearch/BulkProcessorIndexer.java | 15 +- .../elasticsearch/ElasticsearchSinkBase.java | 236 +++++--- ...reHandler.java => NoOpFailureHandler.java} | 6 +- .../RetryRejectedExecutionFailureHandler.java | 46 ++ .../ElasticsearchSinkBaseTest.java | 570 ++++++++++++++++++ 7 files changed, 816 insertions(+), 94 deletions(-) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/{NoOpActionRequestFailureHandler.java => NoOpFailureHandler.java} (85%) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 81652c45..32327ff1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -66,6 +66,14 @@ under the License. test + + org.apache.flink + flink-runtime_2.10 + ${project.version} + test-jar + test + + org.apache.flink flink-streaming-java_2.10 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index 45d04fcb..abbdd727 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -23,7 +23,7 @@ /** * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how failed - * {@link ActionRequest ActionRequests} should be handled, ex. dropping them, reprocessing malformed documents, or + * {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing malformed documents, or * simply requesting them to be sent to Elasticsearch again if the failure is only temporary. * *

@@ -34,19 +34,16 @@ * private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler { * * @Override - * boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer) { - * // this example uses Apache Commons to search for nested exceptions - * - * if (ExceptionUtils.indexOfThrowable(failure, EsRejectedExecutionException.class) >= 0) { + * void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + * if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { * // full queue; re-add document for indexing * indexer.add(action); - * return false; - * } else if (ExceptionUtils.indexOfThrowable(failure, ElasticsearchParseException.class) { + * } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { * // malformed document; simply drop request without failing sink - * return false; * } else { - * // for all other failures, fail the sink - * return true; + * // for all other failures, fail the sink; + * // here the failure is simply rethrown, but users can also choose to throw custom exceptions + * throw failure; * } * } * } @@ -56,6 +53,11 @@ *

* The above example will let the sink re-add requests that failed due to queue capacity saturation and drop requests * with malformed documents, without failing the sink. For all other failures, the sink will fail. + * + *

+ * Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the exact type + * could not be retrieved through the older version Java client APIs (thus, the types will be general {@link Exception}s + * and only differ in the failure message). In this case, it is recommended to match on the provided REST status code. */ public interface ActionRequestFailureHandler extends Serializable { @@ -64,9 +66,12 @@ public interface ActionRequestFailureHandler extends Serializable { * * @param action the {@link ActionRequest} that failed due to the failure * @param failure the cause of failure + * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved) * @param indexer request indexer to re-add the failed action, if intended to do so - * @return the implementation should return {@code true} if the sink should fail due to this failure, and {@code false} otherwise + * + * @throws Throwable if the sink should fail on this failure, the implementation should rethrow + * the exception or a custom one */ - boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer); + void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java index d8025501..838865aa 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -21,6 +21,10 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. @@ -30,14 +34,21 @@ class BulkProcessorIndexer implements RequestIndexer { private static final long serialVersionUID = 6841162943062034253L; private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; - BulkProcessorIndexer(BulkProcessor bulkProcessor) { - this.bulkProcessor = bulkProcessor; + BulkProcessorIndexer(BulkProcessor bulkProcessor, boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); } @Override public void add(ActionRequest... actionRequests) { for (ActionRequest actionRequest : actionRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } this.bulkProcessor.add(actionRequest); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 2c29865f..f6944b3c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -17,8 +17,12 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.action.ActionRequest; @@ -30,11 +34,13 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import static org.apache.flink.util.Preconditions.checkArgument; @@ -56,7 +62,7 @@ * * @param Type of the elements handled by this sink */ -public abstract class ElasticsearchSinkBase extends RichSinkFunction { +public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { private static final long serialVersionUID = -1007596293618451942L; @@ -105,12 +111,12 @@ public void setBackoffType(FlushBackoffType backoffType) { } public void setMaxRetryCount(int maxRetryCount) { - checkArgument(maxRetryCount > 0); + checkArgument(maxRetryCount >= 0); this.maxRetryCount = maxRetryCount; } public void setDelayMillis(long delayMillis) { - checkArgument(delayMillis > 0); + checkArgument(delayMillis >= 0); this.delayMillis = delayMillis; } } @@ -133,6 +139,9 @@ public void setDelayMillis(long delayMillis) { /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ private final ActionRequestFailureHandler failureHandler; + /** If true, the producer will wait until all outstanding action requests have been sent to Elasticsearch. */ + private boolean flushOnCheckpoint = true; + /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient BulkProcessorIndexer requestIndexer; @@ -143,6 +152,17 @@ public void setDelayMillis(long delayMillis) { /** Call bridge for different version-specfic */ private final ElasticsearchApiCallBridge callBridge; + /** + * Number of pending action requests not yet acknowledged by Elasticsearch. + * This value is maintained only if {@link ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}. + * + * This is incremented whenever the user adds (or re-adds through the {@link ActionRequestFailureHandler}) requests + * to the {@link RequestIndexer}. It is decremented for each completed request of a bulk request, in + * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, BulkResponse)} and + * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, Throwable)}. + */ + private AtomicLong numPendingRequests = new AtomicLong(0); + /** Elasticsearch client created using the call bridge. */ private transient Client client; @@ -152,7 +172,7 @@ public void setDelayMillis(long delayMillis) { /** * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown in callbacks and * the user considered it should fail the sink via the - * {@link ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, RequestIndexer)} method. + * {@link ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. * * Errors will be checked and rethrown before processing each input element, and when the sink is closed. */ @@ -172,21 +192,13 @@ public ElasticsearchSinkBase( // otherwise, if they aren't serializable, users will merely get a non-informative error message // "ElasticsearchSinkBase is not serializable" - try { - InstantiationUtil.serializeObject(elasticsearchSinkFunction); - } catch (Exception e) { - throw new IllegalArgumentException( - "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + - "The object probably contains or references non serializable fields."); - } + checkArgument(InstantiationUtil.isSerializable(elasticsearchSinkFunction), + "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + + "The object probably contains or references non-serializable fields."); - try { - InstantiationUtil.serializeObject(failureHandler); - } catch (Exception e) { - throw new IllegalArgumentException( - "The implementation of the provided ActionRequestFailureHandler is not serializable. " + - "The object probably contains or references non serializable fields."); - } + checkArgument(InstantiationUtil.isSerializable(failureHandler), + "The implementation of the provided ActionRequestFailureHandler is not serializable. " + + "The object probably contains or references non-serializable fields."); // extract and remove bulk processor related configuration from the user-provided config, // so that the resulting user config only contains configuration related to the Elasticsearch client. @@ -244,47 +256,76 @@ public ElasticsearchSinkBase( this.userConfig = userConfig; } + /** + * Disable flushing on checkpoint. When disabled, the sink will not wait for all + * pending action requests to be acknowledged by Elasticsearch on checkpoints. + * + * NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT + * provide any strong guarantees for at-least-once delivery of action requests. + */ + public void disableFlushOnCheckpoint() { + this.flushOnCheckpoint = false; + } + @Override public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); + bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); + requestIndexer = new BulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); + } - BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder( - client, - new BulkProcessor.Listener() { - @Override - public void beforeBulk(long executionId, BulkRequest request) { } - - @Override - public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { - if (response.hasFailures()) { - BulkItemResponse itemResponse; - Throwable failure; - - for (int i = 0; i < response.getItems().length; i++) { - itemResponse = response.getItems()[i]; - failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); - if (failure != null) { - LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); - if (failureHandler.onFailure(request.requests().get(i), failure, requestIndexer)) { - failureThrowable.compareAndSet(null, failure); - } - } - } - } - } + @Override + public void invoke(T value) throws Exception { + // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink + checkErrorAndRethrow(); - @Override - public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure.getCause()); + elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + } - // whole bulk request failures are usually just temporary timeouts on - // the Elasticsearch side; simply retry all action requests in the bulk - for (ActionRequest action : request.requests()) { - requestIndexer.add(action); - } - } - } - ); + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // no initialization needed + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + checkErrorAndRethrow(); + + if (flushOnCheckpoint) { + do { + bulkProcessor.flush(); + checkErrorAndRethrow(); + } while (numPendingRequests.get() != 0); + } + } + + @Override + public void close() throws Exception { + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + client = null; + } + + callBridge.cleanup(); + + // make sure any errors from callbacks are rethrown + checkErrorAndRethrow(); + } + + /** + * Build the {@link BulkProcessor}. + * + * Note: this is exposed for testing purposes. + */ + protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { + checkNotNull(listener); + + BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(client, listener); // This makes flush() blocking bulkProcessorBuilder.setConcurrentRequests(0); @@ -304,40 +345,81 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null callBridge.configureBulkProcessorBackoff(bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); - bulkProcessor = bulkProcessorBuilder.build(); - requestIndexer = new BulkProcessorIndexer(bulkProcessor); + return bulkProcessorBuilder.build(); } - @Override - public void invoke(T value) throws Exception { - // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink - checkErrorAndRethrow(); - - elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + private void checkErrorAndRethrow() { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in ElasticsearchSink.", cause); + } } - @Override - public void close() throws Exception { - if (bulkProcessor != null) { - bulkProcessor.close(); - bulkProcessor = null; - } + private class BulkProcessorListener implements BulkProcessor.Listener { + @Override + public void beforeBulk(long executionId, BulkRequest request) { } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + if (response.hasFailures()) { + BulkItemResponse itemResponse; + Throwable failure; + RestStatus restStatus; + + try { + for (int i = 0; i < response.getItems().length; i++) { + itemResponse = response.getItems()[i]; + failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); + if (failure != null) { + LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); + + restStatus = itemResponse.getFailure().getStatus(); + if (restStatus == null) { + failureHandler.onFailure(request.requests().get(i), failure, -1, requestIndexer); + } else { + failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), requestIndexer); + } + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + } - if (client != null) { - client.close(); - client = null; + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } } - callBridge.cleanup(); + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure.getCause()); - // make sure any errors from callbacks are rethrown - checkErrorAndRethrow(); + try { + for (ActionRequest action : request.requests()) { + failureHandler.onFailure(action, failure, -1, requestIndexer); + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } } - private void checkErrorAndRethrow() { - Throwable cause = failureThrowable.get(); - if (cause != null) { - throw new RuntimeException("An error occured in ElasticsearchSink.", cause); + @VisibleForTesting + long getNumPendingRequests() { + if (flushOnCheckpoint) { + return numPendingRequests.get(); + } else { + throw new UnsupportedOperationException( + "The number of pending requests is not maintained when flushing on checkpoint is disabled."); } } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java similarity index 85% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java index 09173a2f..b19ea08b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -24,14 +24,14 @@ /** * An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */ -public class NoOpActionRequestFailureHandler implements ActionRequestFailureHandler { +public class NoOpFailureHandler implements ActionRequestFailureHandler { private static final long serialVersionUID = 737941343410827885L; @Override - public boolean onFailure(ActionRequest action, Throwable failure, RequestIndexer indexer) { + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { // simply fail the sink - return true; + throw failure; } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java new file mode 100644 index 00000000..fabdcbc2 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -0,0 +1,46 @@ +/* + * 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.connectors.elasticsearch.util; + +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.ExceptionUtils; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; + +/** + * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary + * {@link EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), + * and fails for all other failures. + */ +public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = -7423562912824511906L; + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + indexer.add(action); + } else { + // rethrow all other failures + throw failure; + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java new file mode 100644 index 00000000..b9df5c6f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -0,0 +1,570 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.MultiShotLatch; +import org.apache.flink.streaming.api.operators.StreamSink; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.Requests; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.HashMap; +import java.util.List; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** + * Suite of tests for {@link ElasticsearchSinkBase}. + */ +public class ElasticsearchSinkBaseTest { + + /** Tests that any item failure in the listener callbacks is rethrown on an immediately following invoke call. */ + @Test + public void testItemFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the failure + Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** Tests that any item failure in the listener callbacks is rethrown on an immediately following checkpoint. */ + @Test + public void testItemFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the failure + Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any item failure in the listener callbacks due to flushing on an immediately following checkpoint + * is rethrown; we set a timeout because the test will not finish if the logic is broken + */ + @Test(timeout=5000) + public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + + List mockResponsesList = new ArrayList<>(2); + mockResponsesList.add(null); // the first request in a bulk will succeed + mockResponsesList.add(new Exception("artificial failure for record")); // the second request in a bulk will fail + sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); + + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request (1 request only, thus should succeed) + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + + CheckedThread snapshotThread = new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // let the snapshot-triggered flush continue (2 records in the bulk, so the 2nd one should fail) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the failure from the 2nd request + Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** Tests that any bulk failure in the listener callbacks is rethrown on an immediately following invoke call. */ + @Test + public void testBulkFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the bulk request failure + Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** Tests that any bulk failure in the listener callbacks is rethrown on an immediately following checkpoint. */ + @Test + public void testBulkFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks due to flushing on an immediately following checkpoint + * is rethrown; we set a timeout because the test will not finish if the logic is broken. + */ + @Test(timeout=5000) + public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + + CheckedThread snapshotThread = new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // for the snapshot-triggered flush, we let the bulk request fail completely + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + + // let the snapshot-triggered flush continue (bulk request should fail completely) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that the sink correctly waits for pending requests (including re-added requests) on checkpoints; + * we set a timeout because the test will not finish if the logic is broken + */ + @Test(timeout=5000) + public void testAtLeastOnceSink() throws Throwable { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new DummyRetryFailureHandler()); // use a failure handler that simply re-adds requests + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures; + // it contains 1 request, which will fail and re-added to the next bulk request + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + CheckedThread snapshotThread = new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + sink.continueFlush(); + + // since the previous flush should have resulted in a request re-add from the failure handler, + // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // current number of pending request should be 1 due to the re-add + Assert.assertEquals(1, sink.getNumPendingRequests()); + + // this time, let the bulk request succeed, so no-more requests are re-added + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); + + sink.continueFlush(); + + // the snapshot should finish with no exceptions + snapshotThread.sync(); + + testHarness.close(); + } + + /** + * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is disabled, + * the snapshot method does indeed finishes without waiting for pending requests; + * we set a timeout because the test will not finish if the logic is broken + */ + @Test(timeout=5000) + public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception { + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap(), new SimpleSinkFunction(), new DummyRetryFailureHandler()); + sink.disableFlushOnCheckpoint(); // disable flushing + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + + // the snapshot should not block even though we haven't flushed the bulk request + testHarness.snapshot(1L, 1000L); + + testHarness.close(); + } + + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 5051907841570096991L; + + private transient BulkProcessor mockBulkProcessor; + private transient BulkRequest nextBulkRequest = new BulkRequest(); + private transient MultiShotLatch flushLatch = new MultiShotLatch(); + + private List mockItemFailuresList; + private Throwable nextBulkFailure; + + public DummyElasticsearchSink( + Map userConfig, + ElasticsearchSinkFunction sinkFunction, + ActionRequestFailureHandler failureHandler) { + super(new DummyElasticsearchApiCallBridge(), userConfig, sinkFunction, failureHandler); + } + + /** + * This method is used to mimic a scheduled bulk request; we need to do this + * manually because we are mocking the BulkProcessor + */ + public void manualBulkRequestWithAllPendingRequests() { + flushLatch.trigger(); // let the flush + mockBulkProcessor.flush(); + } + + /** + * On non-manual flushes, i.e. when flush is called in the snapshot method implementation, + * usages need to explicitly call this to allow the flush to continue. This is useful + * to make sure that specific requests get added to the the next bulk request for flushing. + */ + public void continueFlush() { + flushLatch.trigger(); + } + + /** + * Set the list of mock failures to use for the next bulk of item responses. A {@code null} + * means that the response is successful, failed otherwise. + * + * The list is used with corresponding order to the requests in the bulk, i.e. the first + * request uses the response at index 0, the second requests uses the response at index 1, etc. + */ + public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { + this.mockItemFailuresList = mockItemFailuresList; + } + + /** + * Let the next bulk request fail completely with the provided throwable. + * If this is set, the failures list provided with setMockItemFailuresListForNextBulkItemResponses is not respected. + */ + public void setFailNextBulkRequestCompletely(Throwable failure) { + this.nextBulkFailure = failure; + } + + public BulkProcessor getMockBulkProcessor() { + return mockBulkProcessor; + } + + /** + * Override the bulk processor build process to provide a mock implementation, + * but reuse the listener implementation in our mock to test that the listener logic + * works correctly with request flushing logic. + */ + @Override + protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { + this.mockBulkProcessor = mock(BulkProcessor.class); + + when(mockBulkProcessor.add(any(ActionRequest.class))).thenAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + // intercept the request and add it to our mock bulk request + nextBulkRequest.add(invocationOnMock.getArgumentAt(0, ActionRequest.class)); + + return null; + } + }); + + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) throws Throwable { + while (nextBulkRequest.numberOfActions() > 0) { + // wait until we are allowed to continue with the flushing + flushLatch.await(); + + // create a copy of the accumulated mock requests, so that + // re-added requests from the failure handler are included in the next bulk + BulkRequest currentBulkRequest = nextBulkRequest; + nextBulkRequest = new BulkRequest(); + + listener.beforeBulk(123L, currentBulkRequest); + + if (nextBulkFailure == null) { + BulkItemResponse[] mockResponses = new BulkItemResponse[currentBulkRequest.requests().size()]; + for (int i = 0; i < currentBulkRequest.requests().size(); i++) { + Throwable mockItemFailure = mockItemFailuresList.get(i); + + if (mockItemFailure == null) { + // the mock response for the item is success + mockResponses[i] = new BulkItemResponse(i, "opType", mock(ActionResponse.class)); + } else { + // the mock response for the item is failure + mockResponses[i] = new BulkItemResponse(i, "opType", new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); + } + } + + listener.afterBulk(123L, currentBulkRequest, new BulkResponse(mockResponses, 1000L)); + } else { + listener.afterBulk(123L, currentBulkRequest, nextBulkFailure); + } + } + + return null; + } + }).when(mockBulkProcessor).flush(); + + return mockBulkProcessor; + } + } + + private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -4272760730959041699L; + + @Override + public Client createClient(Map clientConfig) { + return mock(Client.class); + } + + @Nullable + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (bulkItemResponse.isFailed()) { + return new Exception(bulkItemResponse.getFailure().getMessage()); + } else { + return null; + } + } + + @Override + public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + // no need for this in the test cases here + } + + @Override + public void cleanup() { + // nothing to cleanup + } + } + + private static class SimpleSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = -176739293659135148L; + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + Map json = new HashMap<>(); + json.put("data", element); + + indexer.add( + Requests.indexRequest() + .index("index") + .type("type") + .id("id") + .source(json) + ); + } + } + + private static class DummyRetryFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 5400023700099200745L; + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + indexer.add(action); + } + } +} From f4d26ff7413630d8c26e4f6cfab98fa44a5eafed Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Fri, 28 Apr 2017 17:04:10 +0200 Subject: [PATCH 005/207] [hotfix] [build] Remove unneeded logback-test.xml files --- .../src/test/resources/logback-test.xml | 30 ------------------- 1 file changed, 30 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml deleted file mode 100644 index 45b3b92f..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/logback-test.xml +++ /dev/null @@ -1,30 +0,0 @@ - - - - - - %d{HH:mm:ss.SSS} [%thread] %-5level %logger{60} %X{sourceThread} - %msg%n - - - - - - - - \ No newline at end of file From 2a77ff331b954b0689f1f30be709862d525d94eb Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Mon, 8 May 2017 09:17:43 +0200 Subject: [PATCH 006/207] Update Flink version to 1.4-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 32327ff1..e1833545 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.3-SNAPSHOT + 1.4-SNAPSHOT .. From b8f5097092a930079b6a8ef2967d81da8386c2da Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Thu, 27 Apr 2017 14:43:18 -0400 Subject: [PATCH 007/207] [FLINK-6414] [build] Use scala.binary.version in place of change-scala-version.sh Use scala.binary.version as defined in the parent POM and remove the script to swap scala version identifiers. This closes #3800 --- .../flink-connector-elasticsearch-base/pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index e1833545..79b4f509 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-connector-elasticsearch-base_2.10 + flink-connector-elasticsearch-base_${scala.binary.version} flink-connector-elasticsearch-base jar @@ -46,7 +46,7 @@ under the License. org.apache.flink - flink-streaming-java_2.10 + flink-streaming-java_${scala.binary.version} ${project.version} provided @@ -61,14 +61,14 @@ under the License. org.apache.flink - flink-test-utils_2.10 + flink-test-utils_${scala.binary.version} ${project.version} test org.apache.flink - flink-runtime_2.10 + flink-runtime_${scala.binary.version} ${project.version} test-jar test @@ -76,7 +76,7 @@ under the License. org.apache.flink - flink-streaming-java_2.10 + flink-streaming-java_${scala.binary.version} ${project.version} test test-jar From 9fd9d7b1cfef50ec2422c4f3be4741c792451077 Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 24 May 2017 23:20:29 +0200 Subject: [PATCH 008/207] [FLINK-6711] Activate strict checkstyle for flink-elasticsearch* --- .../ActionRequestFailureHandler.java | 9 ++---- .../ElasticsearchApiCallBridge.java | 3 +- .../elasticsearch/ElasticsearchSinkBase.java | 28 +++++++++++++------ .../ElasticsearchSinkFunction.java | 7 ++--- .../util/NoOpFailureHandler.java | 1 + .../RetryRejectedExecutionFailureHandler.java | 1 + .../ElasticsearchSinkBaseTest.java | 23 +++++++-------- .../ElasticsearchSinkTestBase.java | 13 +++++---- .../EmbeddedElasticsearchNodeEnvironment.java | 2 +- .../testutils/SourceSinkDataTestKit.java | 1 + 10 files changed, 50 insertions(+), 38 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index abbdd727..3ca1417d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -26,8 +26,7 @@ * {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing malformed documents, or * simply requesting them to be sent to Elasticsearch again if the failure is only temporary. * - *

- * Example: + *

Example: * *

{@code
  *
@@ -50,12 +49,10 @@
  *
  * }
* - *

- * The above example will let the sink re-add requests that failed due to queue capacity saturation and drop requests + *

The above example will let the sink re-add requests that failed due to queue capacity saturation and drop requests * with malformed documents, without failing the sink. For all other failures, the sink will fail. * - *

- * Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the exact type + *

Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the exact type * could not be retrieved through the older version Java client APIs (thus, the types will be general {@link Exception}s * and only differ in the failure message). In this case, it is recommended to match on the provided REST status code. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index b4824328..ce98dfba 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -23,6 +23,7 @@ import org.elasticsearch.client.Client; import javax.annotation.Nullable; + import java.io.Serializable; import java.util.Map; @@ -31,7 +32,7 @@ * This includes calls to create Elasticsearch clients, handle failed item responses, etc. Any incompatible Elasticsearch * Java APIs should be bridged using this interface. * - * Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node + *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index f6944b3c..2ab5a90e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -25,6 +25,7 @@ import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.util.InstantiationUtil; + import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; @@ -49,14 +50,12 @@ /** * Base class for all Flink Elasticsearch Sinks. * - *

- * This class implements the common behaviour across Elasticsearch versions, such as + *

This class implements the common behaviour across Elasticsearch versions, such as * the use of an internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before * sending the requests to the cluster, as well as passing input records to the user provided * {@link ElasticsearchSinkFunction} for processing. * - *

- * The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of + *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. * @@ -80,11 +79,21 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction imple public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; + /** + * Used to control whether the retry delay should increase exponentially or remain constant. + */ public enum FlushBackoffType { CONSTANT, EXPONENTIAL } + /** + * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to resource constraints + * (i.e. the client's internal thread pool is full), the backoff policy decides how long the bulk processor will + * wait before the operation is retried internally. + * + *

This is a proxy for version specific backoff policies. + */ public class BulkFlushBackoffPolicy implements Serializable { private static final long serialVersionUID = -6022851996101826049L; @@ -149,14 +158,14 @@ public void setDelayMillis(long delayMillis) { // Internals for the Flink Elasticsearch Sink // ------------------------------------------------------------------------ - /** Call bridge for different version-specfic */ + /** Call bridge for different version-specific. */ private final ElasticsearchApiCallBridge callBridge; /** * Number of pending action requests not yet acknowledged by Elasticsearch. * This value is maintained only if {@link ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}. * - * This is incremented whenever the user adds (or re-adds through the {@link ActionRequestFailureHandler}) requests + *

This is incremented whenever the user adds (or re-adds through the {@link ActionRequestFailureHandler}) requests * to the {@link RequestIndexer}. It is decremented for each completed request of a bulk request, in * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, BulkResponse)} and * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, Throwable)}. @@ -174,7 +183,7 @@ public void setDelayMillis(long delayMillis) { * the user considered it should fail the sink via the * {@link ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. * - * Errors will be checked and rethrown before processing each input element, and when the sink is closed. + *

Errors will be checked and rethrown before processing each input element, and when the sink is closed. */ private final AtomicReference failureThrowable = new AtomicReference<>(); @@ -260,7 +269,7 @@ public ElasticsearchSinkBase( * Disable flushing on checkpoint. When disabled, the sink will not wait for all * pending action requests to be acknowledged by Elasticsearch on checkpoints. * - * NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT + *

NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT * provide any strong guarantees for at-least-once delivery of action requests. */ public void disableFlushOnCheckpoint() { @@ -320,8 +329,9 @@ public void close() throws Exception { /** * Build the {@link BulkProcessor}. * - * Note: this is exposed for testing purposes. + *

Note: this is exposed for testing purposes. */ + @VisibleForTesting protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { checkNotNull(listener); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index 1e20a0a3..82482040 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; + import org.elasticsearch.action.ActionRequest; import java.io.Serializable; @@ -27,11 +28,9 @@ /** * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream. * - *

- * This is used by sinks to prepare elements for sending them to Elasticsearch. + *

This is used by sinks to prepare elements for sending them to Elasticsearch. * - *

- * Example: + *

Example: * *

{@code
  *					private static class TestElasticSearchSinkFunction implements
diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java
index b19ea08b..dffee20d 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java
@@ -19,6 +19,7 @@
 
 import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler;
 import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
+
 import org.elasticsearch.action.ActionRequest;
 
 /**
diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
index fabdcbc2..93809599 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java
@@ -21,6 +21,7 @@
 import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler;
 import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
 import org.apache.flink.util.ExceptionUtils;
+
 import org.elasticsearch.action.ActionRequest;
 import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
 
diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java
index b9df5c6f..5e597856 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java
@@ -24,6 +24,7 @@
 import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+
 import org.elasticsearch.action.ActionRequest;
 import org.elasticsearch.action.ActionResponse;
 import org.elasticsearch.action.bulk.BulkItemResponse;
@@ -41,16 +42,16 @@
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Map;
 import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 /**
  * Suite of tests for {@link ElasticsearchSinkBase}.
@@ -123,9 +124,9 @@ public void testItemFailureRethrownOnCheckpoint() throws Throwable {
 
 	/**
 	 * Tests that any item failure in the listener callbacks due to flushing on an immediately following checkpoint
-	 * is rethrown; we set a timeout because the test will not finish if the logic is broken
+	 * is rethrown; we set a timeout because the test will not finish if the logic is broken.
 	 */
-	@Test(timeout=5000)
+	@Test(timeout = 5000)
 	public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable {
 		final DummyElasticsearchSink sink = new DummyElasticsearchSink<>(
 			new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler());
@@ -250,7 +251,7 @@ public void testBulkFailureRethrownOnCheckpoint() throws Throwable {
 	 * Tests that any bulk failure in the listener callbacks due to flushing on an immediately following checkpoint
 	 * is rethrown; we set a timeout because the test will not finish if the logic is broken.
 	 */
-	@Test(timeout=5000)
+	@Test(timeout = 5000)
 	public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable {
 		final DummyElasticsearchSink sink = new DummyElasticsearchSink<>(
 			new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler());
@@ -307,9 +308,9 @@ public void go() throws Exception {
 
 	/**
 	 * Tests that the sink correctly waits for pending requests (including re-added requests) on checkpoints;
-	 * we set a timeout because the test will not finish if the logic is broken
+	 * we set a timeout because the test will not finish if the logic is broken.
 	 */
-	@Test(timeout=5000)
+	@Test(timeout = 5000)
 	public void testAtLeastOnceSink() throws Throwable {
 		final DummyElasticsearchSink sink = new DummyElasticsearchSink<>(
 				new HashMap(),
@@ -365,9 +366,9 @@ public void go() throws Exception {
 	/**
 	 * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is disabled,
 	 * the snapshot method does indeed finishes without waiting for pending requests;
-	 * we set a timeout because the test will not finish if the logic is broken
+	 * we set a timeout because the test will not finish if the logic is broken.
 	 */
-	@Test(timeout=5000)
+	@Test(timeout = 5000)
 	public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception {
 		final DummyElasticsearchSink sink = new DummyElasticsearchSink<>(
 			new HashMap(), new SimpleSinkFunction(), new DummyRetryFailureHandler());
@@ -409,7 +410,7 @@ public DummyElasticsearchSink(
 
 		/**
 		 * This method is used to mimic a scheduled bulk request; we need to do this
-		 * manually because we are mocking the BulkProcessor
+		 * manually because we are mocking the BulkProcessor.
 		 */
 		public void manualBulkRequestWithAllPendingRequests() {
 			flushLatch.trigger(); // let the flush
@@ -429,7 +430,7 @@ public void continueFlush() {
 		 * Set the list of mock failures to use for the next bulk of item responses. A {@code null}
 		 * means that the response is successful, failed otherwise.
 		 *
-		 * The list is used with corresponding order to the requests in the bulk, i.e. the first
+		 * 

The list is used with corresponding order to the requests in the bulk, i.e. the first * request uses the response at index 0, the second requests uses the response at index 1, etc. */ public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 2f9e4c17..297bc5de 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; import org.apache.flink.util.InstantiationUtil; + import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; import org.junit.AfterClass; @@ -45,7 +46,7 @@ */ public abstract class ElasticsearchSinkTestBase extends StreamingMultipleProgramsTestBase { - protected final static String CLUSTER_NAME = "test-cluster"; + protected static final String CLUSTER_NAME = "test-cluster"; protected static EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; @@ -116,7 +117,7 @@ public void runNullTransportClientTest() throws Exception { try { createElasticsearchSink(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch(IllegalArgumentException expectedException) { + } catch (IllegalArgumentException expectedException) { // test passes return; } @@ -137,7 +138,7 @@ public void runEmptyTransportClientTest() throws Exception { userConfig, Collections.emptyList(), new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch(IllegalArgumentException expectedException) { + } catch (IllegalArgumentException expectedException) { // test passes return; } @@ -162,7 +163,7 @@ public void runTransportClientFailsTest() throws Exception { try { env.execute("Elasticsearch Transport Client Test"); - } catch(JobExecutionException expectedException) { + } catch (JobExecutionException expectedException) { assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); return; } @@ -170,7 +171,7 @@ public void runTransportClientFailsTest() throws Exception { fail(); } - /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses */ + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ protected abstract ElasticsearchSinkBase createElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction); @@ -178,7 +179,7 @@ protected abstract ElasticsearchSinkBase createElasticsearchSink(MapThis case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(Map, List, ElasticsearchSinkFunction)} * because the Elasticsearch Java API to do so is incompatible across different versions. */ protected abstract ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java index f59eb03c..ea6e7a3a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -24,7 +24,7 @@ /** * The {@link EmbeddedElasticsearchNodeEnvironment} is used in integration tests to manage Elasticsearch embedded nodes. * - * NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific implementations + *

NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific implementations * for the tests, concrete implementations must be named {@code EmbeddedElasticsearchNodeEnvironmentImpl}. It must * also be located under the same package. The intentional package-private accessibility of this interface * enforces that. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 55a48fae..4e3d3e2c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; From b9dcc2baac14671732dc93f1a29d9166abc2ca3e Mon Sep 17 00:00:00 2001 From: adebski Date: Sat, 8 Jul 2017 17:03:18 +0200 Subject: [PATCH 009/207] [FLINK-7133] Exclude optional asm deps from Elasticsearch base module These ASM dependencies where shaded into the elasticsearch-base module which where then clashing with our newer (also shaded) ASM dependency. --- .../flink-connector-elasticsearch-base/pom.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 79b4f509..04c254fb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -55,6 +55,19 @@ under the License. org.elasticsearch elasticsearch ${elasticsearch.version} + + + + org.ow2.asm + * + + From 3ad04007d407071a60898f7fbc169279d37d30a1 Mon Sep 17 00:00:00 2001 From: Stephan Ewen Date: Tue, 11 Jul 2017 13:01:25 +0200 Subject: [PATCH 010/207] [FLINK-7150] [elasticsearch connector] Various code cleanups in the ElasticSearch connector - Removes Serializable from the RequestIndexer, because they are neither required to be serializable (they are created in open()) nor is the main implementation (BulkProcessorIndexer) actually serializable. - Makes BulkFlushBackoffPolicy a static inner class, which avoids adding outer class during serialization and clears various warnings about raw reference to outer class This closes #4298 --- .../connectors/elasticsearch/BulkProcessorIndexer.java | 2 -- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 2 +- .../streaming/connectors/elasticsearch/RequestIndexer.java | 4 +--- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java index 838865aa..3e290ff2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -31,8 +31,6 @@ */ class BulkProcessorIndexer implements RequestIndexer { - private static final long serialVersionUID = 6841162943062034253L; - private final BulkProcessor bulkProcessor; private final boolean flushOnCheckpoint; private final AtomicLong numPendingRequestsRef; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 2ab5a90e..c49d7264 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -94,7 +94,7 @@ public enum FlushBackoffType { * *

This is a proxy for version specific backoff policies. */ - public class BulkFlushBackoffPolicy implements Serializable { + public static class BulkFlushBackoffPolicy implements Serializable { private static final long serialVersionUID = -6022851996101826049L; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index 4587a807..cfa166ef 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -20,13 +20,11 @@ import org.elasticsearch.action.ActionRequest; -import java.io.Serializable; - /** * Users add multiple {@link ActionRequest ActionRequests} to a {@link RequestIndexer} to prepare * them for sending to an Elasticsearch cluster. */ -public interface RequestIndexer extends Serializable { +public interface RequestIndexer { /** * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to Elasticsearch. From 9b7e48c706e16b41b29e3cb1588798a5973b8197 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 10 Aug 2017 10:56:12 +0200 Subject: [PATCH 011/207] [FLINK-7409] [web] Make WebRuntimeMonitor reactive This commit changes the behaviour of the WebRuntimeMonitor to not longer block serving threads by waiting on the result of futures. Instead the RequestHandler now returns a CompletableFuture which is written out to the Netty channel upon completion. This will improve the performance of our WebRuntimeMonitor. This closes #4527. --- .../util/RetryRejectedExecutionFailureHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index 93809599..37062571 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -36,7 +36,7 @@ public class RetryRejectedExecutionFailureHandler implements ActionRequestFailur @Override public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { indexer.add(action); } else { // rethrow all other failures From 367ef370028e81a3a240c24f6be9f0adb718ae49 Mon Sep 17 00:00:00 2001 From: yew1eb Date: Fri, 20 Oct 2017 10:45:11 +0800 Subject: [PATCH 012/207] [hotfix][docs][javadocs] Remove double "the" This closes #4865. --- .../connectors/elasticsearch/ElasticsearchSinkBaseTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 5e597856..37e77794 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -420,7 +420,7 @@ public void manualBulkRequestWithAllPendingRequests() { /** * On non-manual flushes, i.e. when flush is called in the snapshot method implementation, * usages need to explicitly call this to allow the flush to continue. This is useful - * to make sure that specific requests get added to the the next bulk request for flushing. + * to make sure that specific requests get added to the next bulk request for flushing. */ public void continueFlush() { flushLatch.trigger(); From e9d74dd7638b60e408f6fad7f6b0e7a2421b5919 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 6 Nov 2017 20:00:08 +0100 Subject: [PATCH 013/207] Update version to 1.5-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 04c254fb..9694786f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.4-SNAPSHOT + 1.5-SNAPSHOT .. From 4d372d3f7e51e63e5e3d0c5bb5b4b404c06fdabc Mon Sep 17 00:00:00 2001 From: Greg Hogan Date: Wed, 3 Jan 2018 14:19:58 -0500 Subject: [PATCH 014/207] [hotfix] Fix many many typos Fix typos from the IntelliJ "Typos" inspection. This closes #5242 --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 2 +- .../elasticsearch/testutils/SourceSinkDataTestKit.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index c49d7264..fe4343ff 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -142,7 +142,7 @@ public void setDelayMillis(long delayMillis) { /** The user specified config map that we forward to Elasticsearch when we create the {@link Client}. */ private final Map userConfig; - /** The function that is used to construct mulitple {@link ActionRequest ActionRequests} from each incoming element. */ + /** The function that is used to construct multiple {@link ActionRequest ActionRequests} from each incoming element. */ private final ElasticsearchSinkFunction elasticsearchSinkFunction; /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 4e3d3e2c..32498c6c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -67,7 +67,7 @@ public void cancel() { } /** - * A {@link ElasticsearchSinkFunction} that indexes each element it receives to a sepecified Elasticsearch index. + * A {@link ElasticsearchSinkFunction} that indexes each element it receives to a specified Elasticsearch index. */ public static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { private static final long serialVersionUID = 1L; From 8909dfd0a921b8b4f40f8addef358b46df0ce54f Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 24 Oct 2017 16:20:15 +0200 Subject: [PATCH 015/207] [FLINK-7909] Replace StreamingMultipleProgramsTestBase by AbstractTestBase The AbstractTestBase fully subsumes the functionality of the StreamingMultipleProgramsTestBase since it now is the most general test base for streaming and batch jobs. As a consequence, we can safely remove the StreamingMultipleProgramsTestBase and let all corresponding tests extend from AbstractTestBase. This closes #4896. --- .../elasticsearch/ElasticsearchSinkTestBase.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 297bc5de..b90e8edf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -22,7 +22,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; -import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase; +import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.client.Client; @@ -31,6 +31,8 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; import java.util.Collections; @@ -44,7 +46,9 @@ /** * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} implementations. */ -public abstract class ElasticsearchSinkTestBase extends StreamingMultipleProgramsTestBase { +public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkTestBase.class); protected static final String CLUSTER_NAME = "test-cluster"; From c925a48369bae1bf7c3b9c97c20ca6cb03f6a3eb Mon Sep 17 00:00:00 2001 From: zhangminglei Date: Mon, 18 Dec 2017 15:52:03 +0800 Subject: [PATCH 016/207] [FLINK-8199] [elasticsearch] Properly annotate APIs of Elasticsearch connector This closes #5124. --- .../connectors/elasticsearch/ActionRequestFailureHandler.java | 3 +++ .../connectors/elasticsearch/BulkProcessorIndexer.java | 3 +++ .../connectors/elasticsearch/ElasticsearchApiCallBridge.java | 3 +++ .../connectors/elasticsearch/ElasticsearchSinkBase.java | 2 ++ .../connectors/elasticsearch/ElasticsearchSinkFunction.java | 2 ++ .../streaming/connectors/elasticsearch/RequestIndexer.java | 3 +++ .../connectors/elasticsearch/util/ElasticsearchUtils.java | 3 +++ .../connectors/elasticsearch/util/NoOpFailureHandler.java | 2 ++ .../util/RetryRejectedExecutionFailureHandler.java | 2 ++ 9 files changed, 23 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index 3ca1417d..260f80e5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -17,6 +17,8 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.PublicEvolving; + import org.elasticsearch.action.ActionRequest; import java.io.Serializable; @@ -56,6 +58,7 @@ * could not be retrieved through the older version Java client APIs (thus, the types will be general {@link Exception}s * and only differ in the failure message). In this case, it is recommended to match on the provided REST status code. */ +@PublicEvolving public interface ActionRequestFailureHandler extends Serializable { /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java index 3e290ff2..2ebb97c8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.Internal; + import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; @@ -29,6 +31,7 @@ * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. */ +@Internal class BulkProcessorIndexer implements RequestIndexer { private final BulkProcessor bulkProcessor; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index ce98dfba..2a7a2165 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.Internal; + import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.Client; @@ -36,6 +38,7 @@ * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. */ +@Internal public interface ElasticsearchApiCallBridge extends Serializable { /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index fe4343ff..d3e0e876 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; @@ -61,6 +62,7 @@ * * @param Type of the elements handled by this sink */ +@Internal public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { private static final long serialVersionUID = -1007596293618451942L; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index 82482040..1b5ce1e0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.RuntimeContext; @@ -56,6 +57,7 @@ * * @param The type of the element handled by this {@code ElasticsearchSinkFunction} */ +@PublicEvolving public interface ElasticsearchSinkFunction extends Serializable, Function { /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index cfa166ef..2a1b2973 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -18,12 +18,15 @@ package org.apache.flink.streaming.connectors.elasticsearch; +import org.apache.flink.annotation.PublicEvolving; + import org.elasticsearch.action.ActionRequest; /** * Users add multiple {@link ActionRequest ActionRequests} to a {@link RequestIndexer} to prepare * them for sending to an Elasticsearch cluster. */ +@PublicEvolving public interface RequestIndexer { /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java index 9776c4c9..11eede4b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java @@ -18,6 +18,8 @@ package org.apache.flink.streaming.connectors.elasticsearch.util; +import org.apache.flink.annotation.Internal; + import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.TransportAddress; @@ -28,6 +30,7 @@ /** * Suite of utility methods for Elasticsearch. */ +@Internal public class ElasticsearchUtils { /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java index dffee20d..dfcb9ee8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.util; +import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; @@ -25,6 +26,7 @@ /** * An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */ +@Internal public class NoOpFailureHandler implements ActionRequestFailureHandler { private static final long serialVersionUID = 737941343410827885L; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index 37062571..ca710cb7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.util; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.ExceptionUtils; @@ -30,6 +31,7 @@ * {@link EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), * and fails for all other failures. */ +@PublicEvolving public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { private static final long serialVersionUID = -7423562912824511906L; From a7192b19b176707b68556248666bb418cace3052 Mon Sep 17 00:00:00 2001 From: zentol Date: Mon, 29 Jan 2018 11:46:08 +0100 Subject: [PATCH 017/207] [FLINK-8489][ES] Prevent side-effects when modifying user-config This closes #5378. This closes #4847. This closes #5305. This closes #5208. This closes #2192. This closes #2422. This closes #3478. --- .../elasticsearch/ElasticsearchSinkBase.java | 4 ++++ .../ElasticsearchSinkBaseTest.java | 20 +++++++++++++++++++ .../ElasticsearchSinkTestBase.java | 2 +- 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index d3e0e876..9105d994 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory; import java.io.Serializable; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -216,6 +217,9 @@ public ElasticsearchSinkBase( checkNotNull(userConfig); + // copy config so we can remove entries without side-effects + userConfig = new HashMap<>(userConfig); + ParameterTool params = ParameterTool.fromMap(userConfig); if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 37e77794..09d8806b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -58,6 +58,26 @@ */ public class ElasticsearchSinkBaseTest { + /** + * Verifies that the collection given to the sink is not modified. + */ + @Test + public void testCollectionArgumentNotModified() { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, "true"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, "CONSTANT"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, "1"); + + new DummyElasticsearchSink<>( + Collections.unmodifiableMap(userConfig), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + } + /** Tests that any item failure in the listener callbacks is rethrown on an immediately following invoke call. */ @Test public void testItemFailureRethrownOnInvoke() throws Throwable { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index b90e8edf..df3779b1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -163,7 +163,7 @@ public void runTransportClientFailsTest() throws Exception { userConfig.put("cluster.name", "my-transport-client-cluster"); source.addSink(createElasticsearchSinkForEmbeddedNode( - userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + Collections.unmodifiableMap(userConfig), new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); try { env.execute("Elasticsearch Transport Client Test"); From fcf3456ea113e9ae182a276091dd53c10b3c7f2d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 27 Feb 2018 14:44:35 +0100 Subject: [PATCH 018/207] Update version to 1.6-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 9694786f..4b4c93c6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.5-SNAPSHOT + 1.6-SNAPSHOT .. From d91d39ba37b984e3347d32038323cac0c2c6cbc7 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 16 Jul 2018 23:24:31 +0200 Subject: [PATCH 019/207] Update version to 1.7-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 4b4c93c6..abb8fbfe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.6-SNAPSHOT + 1.7-SNAPSHOT .. From 889002c3e971034fcdaa12130ce5c47464f6cf24 Mon Sep 17 00:00:00 2001 From: Christophe Jolif Date: Fri, 18 May 2018 00:17:04 +0200 Subject: [PATCH 020/207] [FLINK-7386] [elasticsearch] Evolve ES connector API to make it working with Elasticsearch 5.3+ This closes #6043. --- .../elasticsearch/BulkProcessorIndexer.java | 29 +++++++++++-- .../ElasticsearchApiCallBridge.java | 18 ++++---- .../elasticsearch/ElasticsearchSinkBase.java | 4 +- .../elasticsearch/RequestIndexer.java | 42 ++++++++++++++++++- .../ElasticsearchSinkBaseTest.java | 37 ++++++++-------- 5 files changed, 98 insertions(+), 32 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java index 2ebb97c8..33b42cb4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -22,6 +22,9 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; import java.util.concurrent.atomic.AtomicLong; @@ -45,12 +48,32 @@ class BulkProcessorIndexer implements RequestIndexer { } @Override - public void add(ActionRequest... actionRequests) { - for (ActionRequest actionRequest : actionRequests) { + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { if (flushOnCheckpoint) { numPendingRequestsRef.getAndIncrement(); } - this.bulkProcessor.add(actionRequest); + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); } } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 2a7a2165..1c501bf4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -22,7 +22,6 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.client.Client; import javax.annotation.Nullable; @@ -39,15 +38,18 @@ * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. */ @Internal -public interface ElasticsearchApiCallBridge extends Serializable { +public abstract class ElasticsearchApiCallBridge implements Serializable { /** - * Creates an Elasticsearch {@link Client}. + * Creates an Elasticsearch client implementing {@link AutoCloseable}. This can + * be a {@link org.elasticsearch.client.Client} or {@link org.elasticsearch.client.RestHighLevelClient} * * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - Client createClient(Map clientConfig); + public abstract AutoCloseable createClient(Map clientConfig); + + public abstract BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener); /** * Extracts the cause of failure of a bulk item action. @@ -55,7 +57,7 @@ public interface ElasticsearchApiCallBridge extends Serializable { * @param bulkItemResponse the bulk item response to extract cause of failure * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). */ - @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + public abstract @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); /** * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. @@ -64,13 +66,15 @@ public interface ElasticsearchApiCallBridge extends Serializable { * @param builder the {@link BulkProcessor.Builder} to configure. * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). */ - void configureBulkProcessorBackoff( + public abstract void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); /** * Perform any necessary state cleanup. */ - void cleanup(); + public void cleanup() { + // nothing to cleanup by default + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 9105d994..0305ee3d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -176,7 +176,7 @@ public void setDelayMillis(long delayMillis) { private AtomicLong numPendingRequests = new AtomicLong(0); /** Elasticsearch client created using the call bridge. */ - private transient Client client; + private transient AutoCloseable client; /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ private transient BulkProcessor bulkProcessor; @@ -341,7 +341,7 @@ public void close() throws Exception { protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { checkNotNull(listener); - BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(client, listener); + BulkProcessor.Builder bulkProcessorBuilder = callBridge.createBulkProcessorBuilder(client, listener); // This makes flush() blocking bulkProcessorBuilder.setConcurrentRequests(0); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index 2a1b2973..3dc8f879 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -21,9 +21,12 @@ import org.apache.flink.annotation.PublicEvolving; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; /** - * Users add multiple {@link ActionRequest ActionRequests} to a {@link RequestIndexer} to prepare + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare * them for sending to an Elasticsearch cluster. */ @PublicEvolving @@ -33,6 +36,41 @@ public interface RequestIndexer { * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to Elasticsearch. * * @param actionRequests The multiple {@link ActionRequest} to add. + * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} */ - void add(ActionRequest... actionRequests); + @Deprecated + default void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + if (actionRequest instanceof IndexRequest) { + add((IndexRequest) actionRequest); + } else if (actionRequest instanceof DeleteRequest) { + add((DeleteRequest) actionRequest); + } else if (actionRequest instanceof UpdateRequest) { + add((UpdateRequest) actionRequest); + } else { + throw new IllegalArgumentException("RequestIndexer only supports Index, Delete and Update requests"); + } + } + } + + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 09d8806b..5a161a74 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; import org.junit.Assert; @@ -92,7 +93,7 @@ public void testItemFailureRethrownOnInvoke() throws Throwable { // setup the next bulk request, and its mock item failures sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -124,7 +125,7 @@ public void testItemFailureRethrownOnCheckpoint() throws Throwable { // setup the next bulk request, and its mock item failures sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -164,7 +165,7 @@ public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request (1 request only, thus should succeed) sink.manualBulkRequestWithAllPendingRequests(); @@ -172,7 +173,7 @@ public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { // setup the requests to be flushed in the snapshot testHarness.processElement(new StreamRecord<>("msg-2")); testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -217,7 +218,7 @@ public void testBulkFailureRethrownOnInvoke() throws Throwable { // setup the next bulk request, and let the whole bulk request fail sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -249,7 +250,7 @@ public void testBulkFailureRethrownOnCheckpoint() throws Throwable { // setup the next bulk request, and let the whole bulk request fail sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -284,7 +285,7 @@ public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { // setup the next bulk request, and let bulk request succeed sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -292,7 +293,7 @@ public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { // setup the requests to be flushed in the snapshot testHarness.processElement(new StreamRecord<>("msg-2")); testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -346,7 +347,7 @@ public void testAtLeastOnceSink() throws Throwable { // it contains 1 request, which will fail and re-added to the next bulk request sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -402,7 +403,7 @@ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Excepti // setup the next bulk request, and let bulk request succeed sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // the snapshot should not block even though we haven't flushed the bulk request testHarness.snapshot(1L, 1000L); @@ -478,11 +479,11 @@ public BulkProcessor getMockBulkProcessor() { protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { this.mockBulkProcessor = mock(BulkProcessor.class); - when(mockBulkProcessor.add(any(ActionRequest.class))).thenAnswer(new Answer() { + when(mockBulkProcessor.add(any(IndexRequest.class))).thenAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocationOnMock) throws Throwable { // intercept the request and add it to our mock bulk request - nextBulkRequest.add(invocationOnMock.getArgumentAt(0, ActionRequest.class)); + nextBulkRequest.add(invocationOnMock.getArgumentAt(0, IndexRequest.class)); return null; } @@ -530,12 +531,12 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { } } - private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { + private static class DummyElasticsearchApiCallBridge extends ElasticsearchApiCallBridge { private static final long serialVersionUID = -4272760730959041699L; @Override - public Client createClient(Map clientConfig) { + public AutoCloseable createClient(Map clientConfig) { return mock(Client.class); } @@ -550,13 +551,13 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } @Override - public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { - // no need for this in the test cases here + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return null; } @Override - public void cleanup() { - // nothing to cleanup + public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + // no need for this in the test cases here } } From 163917a6e1308afc1fc1a74c3bf695a11cf5b1c4 Mon Sep 17 00:00:00 2001 From: Christophe Jolif Date: Thu, 25 Jan 2018 22:31:57 +0100 Subject: [PATCH 021/207] [FLINK-8101] [elasticsearch] Elasticsearch 6.X REST support --- .../ElasticsearchApiCallBridge.java | 3 +- .../elasticsearch/ElasticsearchSinkBase.java | 3 +- .../ElasticsearchSinkBaseTest.java | 10 +- .../EmbeddedElasticsearchNodeEnvironment.java | 2 +- .../flink-connector-elasticsearch6/pom.xml | 180 ++++++++++++++++++ .../Elasticsearch6ApiCallBridge.java | 110 +++++++++++ .../elasticsearch6/ElasticsearchSink.java | 91 +++++++++ ...eddedElasticsearchNodeEnvironmentImpl.java | 82 ++++++++ .../ElasticsearchSinkITCase.java | 152 +++++++++++++++ .../examples/ElasticsearchSinkExample.java | 81 ++++++++ .../src/test/resources/log4j-test.properties | 27 +++ 11 files changed, 731 insertions(+), 10 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch6/pom.xml create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 1c501bf4..90d84f36 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -41,8 +41,7 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { /** - * Creates an Elasticsearch client implementing {@link AutoCloseable}. This can - * be a {@link org.elasticsearch.client.Client} or {@link org.elasticsearch.client.RestHighLevelClient} + * Creates an Elasticsearch client implementing {@link AutoCloseable}. * * @param clientConfig The configuration to use when constructing the client. * @return The created client. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 0305ee3d..98304844 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -32,7 +32,6 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -59,7 +58,7 @@ * *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, - * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. + * for example, to create a Elasticsearch {@link Client} or {@RestHighLevelClient}, handle failed item responses, etc. * * @param Type of the elements handled by this sink */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 5a161a74..460e939f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -540,6 +540,11 @@ public AutoCloseable createClient(Map clientConfig) { return mock(Client.class); } + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return null; + } + @Nullable @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { @@ -550,11 +555,6 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } } - @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - return null; - } - @Override public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { // no need for this in the test cases here diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java index ea6e7a3a..fd14ba36 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -29,7 +29,7 @@ * also be located under the same package. The intentional package-private accessibility of this interface * enforces that. */ -interface EmbeddedElasticsearchNodeEnvironment { +public interface EmbeddedElasticsearchNodeEnvironment { /** * Start an embedded Elasticsearch node instance. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml new file mode 100644 index 00000000..e453837f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -0,0 +1,180 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.7-SNAPSHOT + .. + + + flink-connector-elasticsearch6_${scala.binary.version} + flink-connector-elasticsearch6 + + jar + + + + 6.3.1 + + + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + + + org.apache.logging.log4j + log4j-to-slf4j + 2.7 + + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + + org.elasticsearch.client + transport + ${elasticsearch.version} + test + + + + org.elasticsearch.plugin + transport-netty4-client + ${elasticsearch.version} + test + + + + + + org.apache.logging.log4j + log4j-api + 2.7 + test + + + + org.apache.logging.log4j + log4j-core + 2.7 + test + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.2 + + + org.apache.logging.log4j:log4j-to-slf4j + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java new file mode 100644 index 00000000..2cb4ea03 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -0,0 +1,110 @@ +/* + * 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.connectors.elasticsearch6; + +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. + */ +public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6ApiCallBridge.class); + + /** + * User-provided HTTP Host. + */ + private final List httpHosts; + + Elasticsearch6ApiCallBridge(List httpHosts) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + } + + @Override + public AutoCloseable createClient(Map clientConfig) { + RestHighLevelClient rhlClient = + new RestHighLevelClient(RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]))); + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); + } + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + RestHighLevelClient rhlClient = (RestHighLevelClient) client; + return BulkProcessor.builder(rhlClient::bulkAsync, listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java new file mode 100644 index 00000000..3f75b5ff --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -0,0 +1,91 @@ +/* + * 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.connectors.elasticsearch6; + +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.List; +import java.util.Map; + +/** + * Elasticsearch 6.x sink that requests multiple {@link ActionRequest ActionRequests} + * against a cluster for each incoming element. + * + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. + * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. + * + *

The {@link Map} passed to the constructor is used to create the {@code TransportClient}. The config keys can be found + * in the Elasticsearch documentation. An important setting is {@code cluster.name}, + * which should be set to the name of the cluster that the sink should emit to. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. + * This will buffer elements before sending a request to the cluster. The behaviour of the + * {@code BulkProcessor} can be configured using these config keys: + *

    + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
+ * + *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of + * {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + */ +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element + * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + */ + public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + + this(userConfig, httpHosts, elasticsearchSinkFunction, new NoOpFailureHandler()); + } + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element + * @param failureHandler This is used to handle failed {@link ActionRequest} + * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + */ + public ElasticsearchSink( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler) { + + super(new Elasticsearch6ApiCallBridge(httpHosts), userConfig, elasticsearchSinkFunction, failureHandler); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java new file mode 100644 index 00000000..f419b41b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -0,0 +1,82 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase; + +import org.elasticsearch.client.Client; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.InternalSettingsPreparer; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.transport.Netty4Plugin; + +import java.io.File; +import java.util.Collections; + +/** + * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 6. + * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. + */ +public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { + + private Node node; + + @Override + public void start(File tmpDataFolder, String clusterName) throws Exception { + if (node == null) { + Settings settings = Settings.builder() + .put("cluster.name", clusterName) + .put("http.enabled", false) + .put("path.home", tmpDataFolder.getParent()) + .put("path.data", tmpDataFolder.getAbsolutePath()) + .put(NetworkModule.HTTP_TYPE_KEY, Netty4Plugin.NETTY_HTTP_TRANSPORT_NAME) + .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4Plugin.NETTY_TRANSPORT_NAME) + .build(); + + node = new PluginNode(settings); + node.start(); + } + } + + @Override + public void close() throws Exception { + if (node != null && !node.isClosed()) { + node.close(); + node = null; + } + } + + @Override + public Client getClient() { + if (node != null && !node.isClosed()) { + return node.client(); + } else { + return null; + } + } + + private static class PluginNode extends Node { + public PluginNode(Settings settings) { + super(InternalSettingsPreparer.prepareEnvironment(settings, null), Collections.>singletonList(Netty4Plugin.class)); + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java new file mode 100644 index 00000000..21707717 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -0,0 +1,152 @@ +/* + * 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.connectors.elasticsearch6; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.RestHighLevelClient; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * IT cases for the {@link ElasticsearchSink}. + */ +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + + /** + * Tests that the Elasticsearch sink works properly using a {@link RestHighLevelClient}. + */ + public void runTransportClientTest() throws Exception { + final String index = "transport-client-test-index"; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + + env.execute("Elasticsearch RestHighLevelClient Test"); + + // verify the results + Client client = embeddedNodeEnv.getClient(); + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. + */ + public void runNullTransportClientTest() throws Exception { + try { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + createElasticsearchSink6(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. + */ + public void runEmptyTransportClientTest() throws Exception { + try { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + createElasticsearchSink6(userConfig, + Collections.emptyList(), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. + */ + public void runTransportClientFailsTest() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + + try { + env.execute("Elasticsearch Transport Client Test"); + } catch (JobExecutionException expectedException) { + assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); + return; + } + + fail(); + } + + @Override + protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { + return null; + } + + @Override + protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + } + + private ElasticsearchSinkBase createElasticsearchSink6( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction) { + return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java new file mode 100644 index 00000000..de1670fe --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java @@ -0,0 +1,81 @@ +/* + * 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.connectors.elasticsearch6.examples; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that + * you have a cluster named "elasticsearch" running or change the name of cluster in the config map. + */ +public class ElasticsearchSinkExample { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.generateSequence(0, 20).map(new MapFunction() { + @Override + public String map(Long value) throws Exception { + return "message #" + value; + } + }); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + List httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + + source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + })); + + env.execute("Elasticsearch Sink Example"); + } + + private static IndexRequest createIndexRequest(String element) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .id(element) + .source(json); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties new file mode 100644 index 00000000..20551848 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger From c0faa8e8111d930a0f6106600501d22ace02fca2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 25 Jul 2018 17:21:34 +0800 Subject: [PATCH 022/207] [FLINK-9885] [elasticsearch] Major cleanup to finalize Elasticsearch 6.x connector This closes #6391. --- .../ElasticsearchApiCallBridge.java | 22 ++- .../elasticsearch/ElasticsearchSinkBase.java | 24 ++- .../ElasticsearchSinkBaseTest.java | 8 +- .../ElasticsearchSinkTestBase.java | 103 +++++++---- .../flink-connector-elasticsearch6/pom.xml | 6 +- .../Elasticsearch6ApiCallBridge.java | 35 +++- .../elasticsearch6/ElasticsearchSink.java | 168 +++++++++++++++--- .../elasticsearch6/RestClientFactory.java | 40 +++++ ...eddedElasticsearchNodeEnvironmentImpl.java | 5 +- .../ElasticsearchSinkITCase.java | 144 +++++---------- .../examples/ElasticsearchSinkExample.java | 81 --------- .../src/test/resources/log4j-test.properties | 3 - 12 files changed, 364 insertions(+), 275 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 90d84f36..f1dcc83f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.io.Serializable; import java.util.Map; @@ -36,9 +37,11 @@ *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. + * + * @param The Elasticsearch client, that implements {@link AutoCloseable}. */ @Internal -public abstract class ElasticsearchApiCallBridge implements Serializable { +public interface ElasticsearchApiCallBridge extends Serializable { /** * Creates an Elasticsearch client implementing {@link AutoCloseable}. @@ -46,9 +49,16 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - public abstract AutoCloseable createClient(Map clientConfig); + C createClient(Map clientConfig) throws IOException; - public abstract BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener); + /** + * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. + * + * @param client the Elasticsearch client. + * @param listener the bulk processor listender. + * @return the bulk processor builder. + */ + BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); /** * Extracts the cause of failure of a bulk item action. @@ -56,7 +66,7 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param bulkItemResponse the bulk item response to extract cause of failure * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). */ - public abstract @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); /** * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. @@ -65,14 +75,14 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param builder the {@link BulkProcessor.Builder} to configure. * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). */ - public abstract void configureBulkProcessorBackoff( + void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); /** * Perform any necessary state cleanup. */ - public void cleanup() { + default void cleanup() { // nothing to cleanup by default } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 98304844..7dac06ce 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; @@ -32,6 +33,7 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -58,12 +60,13 @@ * *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, - * for example, to create a Elasticsearch {@link Client} or {@RestHighLevelClient}, handle failed item responses, etc. + * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. * * @param Type of the elements handled by this sink + * @param Type of the Elasticsearch client, which implements {@link AutoCloseable} */ @Internal -public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { +public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { private static final long serialVersionUID = -1007596293618451942L; @@ -84,6 +87,7 @@ public abstract class ElasticsearchSinkBase extends RichSinkFunction imple /** * Used to control whether the retry delay should increase exponentially or remain constant. */ + @PublicEvolving public enum FlushBackoffType { CONSTANT, EXPONENTIAL @@ -134,14 +138,20 @@ public void setDelayMillis(long delayMillis) { private final Integer bulkProcessorFlushMaxActions; private final Integer bulkProcessorFlushMaxSizeMb; - private final Integer bulkProcessorFlushIntervalMillis; + private final Long bulkProcessorFlushIntervalMillis; private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; // ------------------------------------------------------------------------ // User-facing API and configuration // ------------------------------------------------------------------------ - /** The user specified config map that we forward to Elasticsearch when we create the {@link Client}. */ + /** + * The config map that contains configuration for the bulk flushing behaviours. + * + *

For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this config + * map would also contain Elasticsearch-shipped configuration, and therefore this config map + * would also be forwarded when creating the Elasticsearch client. + */ private final Map userConfig; /** The function that is used to construct multiple {@link ActionRequest ActionRequests} from each incoming element. */ @@ -161,7 +171,7 @@ public void setDelayMillis(long delayMillis) { // ------------------------------------------------------------------------ /** Call bridge for different version-specific. */ - private final ElasticsearchApiCallBridge callBridge; + private final ElasticsearchApiCallBridge callBridge; /** * Number of pending action requests not yet acknowledged by Elasticsearch. @@ -175,7 +185,7 @@ public void setDelayMillis(long delayMillis) { private AtomicLong numPendingRequests = new AtomicLong(0); /** Elasticsearch client created using the call bridge. */ - private transient AutoCloseable client; + private transient C client; /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ private transient BulkProcessor bulkProcessor; @@ -236,7 +246,7 @@ public ElasticsearchSinkBase( } if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { - bulkProcessorFlushIntervalMillis = params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); } else { bulkProcessorFlushIntervalMillis = null; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 460e939f..369d26a7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -411,7 +411,7 @@ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Excepti testHarness.close(); } - private static class DummyElasticsearchSink extends ElasticsearchSinkBase { + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 5051907841570096991L; @@ -531,17 +531,17 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { } } - private static class DummyElasticsearchApiCallBridge extends ElasticsearchApiCallBridge { + private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -4272760730959041699L; @Override - public AutoCloseable createClient(Map clientConfig) { + public Client createClient(Map clientConfig) { return mock(Client.class); } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + public BulkProcessor.Builder createBulkProcessorBuilder(Client client, BulkProcessor.Listener listener) { return null; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index df3779b1..819ffba5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -26,7 +26,6 @@ import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -34,19 +33,20 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} implementations. + * + * @param Elasticsearch client type + * @param The address type to use */ -public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { +public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkTestBase.class); @@ -85,24 +85,21 @@ public static void shutdown() throws Exception { } /** - * Tests that the Elasticsearch sink works properly using a {@link TransportClient}. + * Tests that the Elasticsearch sink works properly. */ - public void runTransportClientTest() throws Exception { - final String index = "transport-client-test-index"; + public void runElasticsearchSinkTest() throws Exception { + final String index = "elasticsearch-sink-test-index"; final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", CLUSTER_NAME); - source.addSink(createElasticsearchSinkForEmbeddedNode( - userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + 1, + CLUSTER_NAME, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); - env.execute("Elasticsearch TransportClient Test"); + env.execute("Elasticsearch Sink Test"); // verify the results Client client = embeddedNodeEnv.getClient(); @@ -112,16 +109,20 @@ public void runTransportClientTest() throws Exception { } /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code null}. */ - public void runNullTransportClientTest() throws Exception { + public void runNullAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", CLUSTER_NAME); try { - createElasticsearchSink(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { + createElasticsearchSink( + 1, + CLUSTER_NAME, + null, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch (IllegalArgumentException | NullPointerException expectedException) { // test passes return; } @@ -130,18 +131,19 @@ public void runNullTransportClientTest() throws Exception { } /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. */ - public void runEmptyTransportClientTest() throws Exception { + public void runEmptyAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", CLUSTER_NAME); try { createElasticsearchSink( - userConfig, - Collections.emptyList(), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + 1, + CLUSTER_NAME, + Collections.emptyList(), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); } catch (IllegalArgumentException expectedException) { // test passes return; @@ -153,39 +155,66 @@ public void runEmptyTransportClientTest() throws Exception { /** * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. */ - public void runTransportClientFailsTest() throws Exception { + public void runInvalidElasticsearchClusterTest() throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", "invalid-cluster-name"); - source.addSink(createElasticsearchSinkForEmbeddedNode( - Collections.unmodifiableMap(userConfig), new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + source.addSink(createElasticsearchSinkForNode( + 1, + "invalid-cluster-name", + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"), + "123.123.123.123")); // incorrect ip address try { - env.execute("Elasticsearch Transport Client Test"); + env.execute("Elasticsearch Sink Test"); } catch (JobExecutionException expectedException) { - assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); + // test passes return; } fail(); } + /** + * Utility method to create a user config map. + */ + protected Map createUserConfig(int bulkFlushMaxActions, String clusterName) { + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", clusterName); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(bulkFlushMaxActions)); + + return userConfig; + } + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ - protected abstract ElasticsearchSinkBase createElasticsearchSink(Map userConfig, - List transportAddresses, - ElasticsearchSinkFunction elasticsearchSinkFunction); + protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List addresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction); /** * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch node. * - *

This case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(Map, List, ElasticsearchSinkFunction)} + *

This case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(int, String, List, ElasticsearchSinkFunction)} * because the Elasticsearch Java API to do so is incompatible across different versions. */ - protected abstract ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception; + protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; + + /** + * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. + */ + protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception; } diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index e453837f..ef06d805 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -81,7 +81,7 @@ under the License. org.apache.logging.log4j log4j-to-slf4j - 2.7 + 2.9.1 @@ -141,14 +141,14 @@ under the License. org.apache.logging.log4j log4j-api - 2.7 + 2.9.1 test org.apache.logging.log4j log4j-core - 2.7 + 2.9.1 test diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 2cb4ea03..03bf9c07 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.elasticsearch6; +import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.util.Preconditions; @@ -26,6 +27,7 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.TimeValue; import org.slf4j.Logger; @@ -33,13 +35,15 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.util.List; import java.util.Map; /** * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ -public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { +@Internal +public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -5222683870097809633L; @@ -50,15 +54,31 @@ public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { */ private final List httpHosts; - Elasticsearch6ApiCallBridge(List httpHosts) { + /** + * The factory to configure the rest client. + */ + private final RestClientFactory restClientFactory; + + Elasticsearch6ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); } @Override - public AutoCloseable createClient(Map clientConfig) { - RestHighLevelClient rhlClient = - new RestHighLevelClient(RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]))); + public RestHighLevelClient createClient(Map clientConfig) throws IOException { + RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!rhlClient.ping()) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); @@ -68,9 +88,8 @@ public AutoCloseable createClient(Map clientConfig) { } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - RestHighLevelClient rhlClient = (RestHighLevelClient) client; - return BulkProcessor.builder(rhlClient::bulkAsync, listener); + public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client::bulkAsync, listener); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 3f75b5ff..4e7a2635 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -17,17 +17,19 @@ package org.apache.flink.streaming.connectors.elasticsearch6; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.RestHighLevelClient; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -38,10 +40,6 @@ *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. * - *

The {@link Map} passed to the constructor is used to create the {@code TransportClient}. The config keys can be found - * in the Elasticsearch documentation. An important setting is {@code cluster.name}, - * which should be set to the name of the cluster that the sink should emit to. - * *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. * This will buffer elements before sending a request to the cluster. The behaviour of the * {@code BulkProcessor} can be configured using these config keys: @@ -58,34 +56,156 @@ * * @param Type of the elements handled by this sink */ -public class ElasticsearchSink extends ElasticsearchSinkBase { +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 1L; - /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. - * - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element - * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. - */ - public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { - this(userConfig, httpHosts, elasticsearchSinkFunction, new NoOpFailureHandler()); + super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); } /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * A builder for creating an {@link ElasticsearchSink}. * - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element - * @param failureHandler This is used to handle failed {@link ActionRequest} - * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param Type of the elements handled by the sink this builder creates. */ - public ElasticsearchSink( - Map userConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler) { + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + */ + public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maxinum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, + "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } - super(new Elasticsearch6ApiCallBridge(httpHosts), userConfig, elasticsearchSinkFunction, failureHandler); + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); + } } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java new file mode 100644 index 00000000..4b74649c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java @@ -0,0 +1,40 @@ +/* + * 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.connectors.elasticsearch6; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally + * used in the {@link ElasticsearchSink}. + */ +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); + +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java index f419b41b..8dc62168 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -21,7 +21,6 @@ import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase; import org.elasticsearch.client.Client; -import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.InternalSettingsPreparer; import org.elasticsearch.node.Node; @@ -44,11 +43,9 @@ public void start(File tmpDataFolder, String clusterName) throws Exception { if (node == null) { Settings settings = Settings.builder() .put("cluster.name", clusterName) - .put("http.enabled", false) + .put("http.enabled", true) .put("path.home", tmpDataFolder.getParent()) .put("path.data", tmpDataFolder.getAbsolutePath()) - .put(NetworkModule.HTTP_TYPE_KEY, Netty4Plugin.NETTY_HTTP_TRANSPORT_NAME) - .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4Plugin.NETTY_TRANSPORT_NAME) .build(); node = new PluginNode(settings); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 21707717..a6f01258 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -19,134 +19,82 @@ package org.apache.flink.streaming.connectors.elasticsearch6; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; -import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.http.HttpHost; -import org.elasticsearch.client.Client; import org.elasticsearch.client.RestHighLevelClient; +import org.junit.Test; -import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; /** * IT cases for the {@link ElasticsearchSink}. + * + *

The Elasticsearch ITCases for 6.x CANNOT be executed in the IDE directly, since it is required that the + * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded + * node used in the tests to work properly. */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - - /** - * Tests that the Elasticsearch sink works properly using a {@link RestHighLevelClient}. - */ - public void runTransportClientTest() throws Exception { - final String index = "transport-client-test-index"; - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - - source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, - new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); - - env.execute("Elasticsearch RestHighLevelClient Test"); - - // verify the results - Client client = embeddedNodeEnv.getClient(); - SourceSinkDataTestKit.verifyProducedSinkData(client, index); +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - client.close(); + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); } - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. - */ - public void runNullTransportClientTest() throws Exception { - try { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - createElasticsearchSink6(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { - // test passes - return; - } - - fail(); + @Test + public void testNullAddresses() throws Exception { + runNullAddressesTest(); } - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. - */ - public void runEmptyTransportClientTest() throws Exception { - try { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - createElasticsearchSink6(userConfig, - Collections.emptyList(), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { - // test passes - return; - } - - fail(); + @Test + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); } - /** - * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. - */ - public void runTransportClientFailsTest() throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + @Test + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); + } - source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { - try { - env.execute("Elasticsearch Transport Client Test"); - } catch (JobExecutionException expectedException) { - assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); - return; - } + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); - fail(); + return builder.build(); } @Override - protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { - return null; + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); } @Override - protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); - return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); - } + httpHosts.add(new HttpHost(ipAddress, 9200, "http")); + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); - private ElasticsearchSinkBase createElasticsearchSink6( - Map userConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + return builder.build(); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java deleted file mode 100644 index de1670fe..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.connectors.elasticsearch6.examples; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; -import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.client.Requests; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that - * you have a cluster named "elasticsearch" running or change the name of cluster in the config map. - */ -public class ElasticsearchSinkExample { - - public static void main(String[] args) throws Exception { - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream source = env.generateSequence(0, 20).map(new MapFunction() { - @Override - public String map(Long value) throws Exception { - return "message #" + value; - } - }); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - - List httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); - - source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element)); - } - })); - - env.execute("Elasticsearch Sink Example"); - } - - private static IndexRequest createIndexRequest(String element) { - Map json = new HashMap<>(); - json.put("data", element); - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .id(element) - .source(json); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties index 20551848..fcd86546 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties @@ -22,6 +22,3 @@ log4j.appender.testlogger=org.apache.log4j.ConsoleAppender log4j.appender.testlogger.target=System.err log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n - -# suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger From 97c013ed6585711237bd214f29ee81b2369b873b Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 12 Sep 2018 12:21:34 +0200 Subject: [PATCH 023/207] [FLINK-10269] [connectors] Fix Elasticsearch 6 UpdateRequest binary incompatibility This commit fixes the binary incompatibility for UpdateRequests in Elasticsearch. This is due to a binary compatibility issue between the base module (which is compiled against a very old ES version and the current Elasticsearch version). It lets the API call bridge also provide the RequestIndexer version-specific. This closes #6682. --- .../ElasticsearchApiCallBridge.java | 14 +++ .../elasticsearch/ElasticsearchSinkBase.java | 4 +- ...PreElasticsearch6BulkProcessorIndexer.java | 84 ++++++++++++++++++ .../Elasticsearch6ApiCallBridge.java | 13 +++ .../Elasticsearch6BulkProcessorIndexer.java | 85 +++++++++++++++++++ 5 files changed, 198 insertions(+), 2 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index f1dcc83f..d3b774c8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; /** * An {@link ElasticsearchApiCallBridge} is used to bridge incompatible Elasticsearch Java API calls across different versions. @@ -79,6 +80,19 @@ void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); + /** + * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. + */ + default RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new PreElasticsearch6BulkProcessorIndexer( + bulkProcessor, + flushOnCheckpoint, + numPendingRequestsRef); + } + /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 7dac06ce..4d0c0025 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -164,7 +164,7 @@ public void setDelayMillis(long delayMillis) { private boolean flushOnCheckpoint = true; /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ - private transient BulkProcessorIndexer requestIndexer; + private transient RequestIndexer requestIndexer; // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink @@ -295,7 +295,7 @@ public void disableFlushOnCheckpoint() { public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); - requestIndexer = new BulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); + requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java new file mode 100644 index 00000000..85f4b9a3 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java @@ -0,0 +1,84 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. + * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + * @deprecated This class is not binary compatible with newer Elasticsearch 6+ versions + * (i.e. the {@link #add(UpdateRequest...)} ). However, this module is currently + * compiled against a very old Elasticsearch version. + */ +@Deprecated +@Internal +class PreElasticsearch6BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + PreElasticsearch6BulkProcessorIndexer(BulkProcessor bulkProcessor, boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 03bf9c07..782cbbcf 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.util.Preconditions; import org.apache.http.HttpHost; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; /** * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. @@ -126,4 +128,15 @@ public void configureBulkProcessorBackoff( builder.setBackoffPolicy(backoffPolicy); } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch6BulkProcessorIndexer( + bulkProcessor, + flushOnCheckpoint, + numPendingRequestsRef); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java new file mode 100644 index 00000000..af3c5b13 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.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 org.apache.flink.streaming.connectors.elasticsearch6; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. + * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + *

Note: This class is binary compatible to Elasticsearch 6. + */ +@Internal +class Elasticsearch6BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + Elasticsearch6BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} From bf20298efb8705fcb349bdf9b2b967198b187a15 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 1 Oct 2018 11:02:05 +0200 Subject: [PATCH 024/207] [hotfix] [connectors] Remove unused BulkProcessorIndexer class --- .../elasticsearch/BulkProcessorIndexer.java | 79 ------------------- 1 file changed, 79 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java deleted file mode 100644 index 33b42cb4..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.annotation.Internal; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; - -import java.util.concurrent.atomic.AtomicLong; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. - * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. - */ -@Internal -class BulkProcessorIndexer implements RequestIndexer { - - private final BulkProcessor bulkProcessor; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; - - BulkProcessorIndexer(BulkProcessor bulkProcessor, boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { - this.bulkProcessor = checkNotNull(bulkProcessor); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } - - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(deleteRequest); - } - } - - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(indexRequest); - } - } - - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(updateRequest); - } - } -} From 05ad4dfd31775767258285d57d3699754a1736f2 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 15 Aug 2018 13:51:23 +0200 Subject: [PATCH 025/207] [FLINK-3875] [connectors] Add an upsert table sink factory for Elasticsearch This commit adds full support for Elasticsearch to be used with Table & SQL API as well as SQL Client. It includes: - Elasticsearch 6 upsert table sink (for append-only and updating queries) - Elasticsearch 6 table factory - Elasticsearch table descriptors & validators - Unit tests, SQL Client end-to-end test - Website documentation This closes #6611. --- .../pom.xml | 27 + .../ElasticsearchUpsertTableSinkBase.java | 522 ++++++++++++++++++ ...asticsearchUpsertTableSinkFactoryBase.java | 296 ++++++++++ .../util/IgnoringFailureHandler.java | 39 ++ .../table/descriptors/Elasticsearch.java | 315 +++++++++++ .../descriptors/ElasticsearchValidator.java | 129 +++++ ...csearchUpsertTableSinkFactoryTestBase.java | 189 +++++++ .../table/descriptors/ElasticsearchTest.java | 147 +++++ .../flink-connector-elasticsearch6/pom.xml | 117 +++- .../Elasticsearch6UpsertTableSink.java | 269 +++++++++ .../Elasticsearch6UpsertTableSinkFactory.java | 77 +++ .../elasticsearch6/ElasticsearchSink.java | 27 + ....apache.flink.table.factories.TableFactory | 16 + ...sticsearch6UpsertTableSinkFactoryTest.java | 223 ++++++++ 14 files changed, 2389 insertions(+), 4 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index abb8fbfe..9fbd9b37 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -70,6 +70,16 @@ under the License. + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + true + + @@ -95,6 +105,23 @@ under the License. test-jar + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java new file mode 100644 index 00000000..9bee8c38 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -0,0 +1,522 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.typeutils.TypeCheckUtils; +import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A version-agnostic Elasticsearch {@link UpsertStreamTableSink}. + */ +@Internal +public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTableSink { + + /** Flag that indicates that only inserts are accepted. */ + private final boolean isAppendOnly; + + /** Schema of the table. */ + private final TableSchema schema; + + /** Version-agnostic hosts configuration. */ + private final List hosts; + + /** Default index for all requests. */ + private final String index; + + /** Default document type for all requests. */ + private final String docType; + + /** Delimiter for composite keys. */ + private final String keyDelimiter; + + /** String literal for null keys. */ + private final String keyNullLiteral; + + /** Serialization schema used for the document. */ + private final SerializationSchema serializationSchema; + + /** Content type describing the serialization schema. */ + private final XContentType contentType; + + /** Failure handler for failing {@link ActionRequest}s. */ + private final ActionRequestFailureHandler failureHandler; + + /** + * Map of optional configuration parameters for the Elasticsearch sink. The config is + * internal and can change at any time. + */ + private final Map sinkOptions; + + /** + * Version-agnostic creation of {@link ActionRequest}s. + */ + private final RequestFactory requestFactory; + + /** Key field indices determined by the query. */ + private int[] keyFieldIndices = new int[0]; + + public ElasticsearchUpsertTableSinkBase( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + this.isAppendOnly = isAppendOnly; + this.schema = Preconditions.checkNotNull(schema); + this.hosts = Preconditions.checkNotNull(hosts); + this.index = Preconditions.checkNotNull(index); + this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); + this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); + this.docType = Preconditions.checkNotNull(docType); + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.failureHandler = Preconditions.checkNotNull(failureHandler); + this.sinkOptions = Preconditions.checkNotNull(sinkOptions); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + } + + @Override + public void setKeyFields(String[] keyNames) { + if (keyNames == null) { + this.keyFieldIndices = new int[0]; + return; + } + + final String[] fieldNames = getFieldNames(); + final int[] keyFieldIndices = new int[keyNames.length]; + for (int i = 0; i < keyNames.length; i++) { + keyFieldIndices[i] = -1; + for (int j = 0; j < fieldNames.length; j++) { + if (keyNames[i].equals(fieldNames[j])) { + keyFieldIndices[i] = j; + break; + } + } + if (keyFieldIndices[i] == -1) { + throw new RuntimeException("Invalid key fields: " + Arrays.toString(keyNames)); + } + } + + validateKeyTypes(keyFieldIndices); + + this.keyFieldIndices = keyFieldIndices; + } + + @Override + public void setIsAppendOnly(Boolean isAppendOnly) { + if (this.isAppendOnly && !isAppendOnly) { + throw new ValidationException( + "The given query is not supported by this sink because the sink is configured to " + + "operate in append mode only. Thus, it only support insertions (no queries " + + "with updating results)."); + } + } + + @Override + public TypeInformation getRecordType() { + return schema.toRowType(); + } + + @Override + public void emitDataStream(DataStream> dataStream) { + final ElasticsearchUpsertSinkFunction upsertFunction = + new ElasticsearchUpsertSinkFunction( + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + requestFactory, + keyFieldIndices); + final SinkFunction> sinkFunction = createSinkFunction( + hosts, + failureHandler, + sinkOptions, + upsertFunction); + dataStream.addSink(sinkFunction) + .name(TableConnectorUtil.generateRuntimeName(this.getClass(), getFieldNames())); + } + + @Override + public TypeInformation> getOutputType() { + return Types.TUPLE(Types.BOOLEAN, getRecordType()); + } + + @Override + public String[] getFieldNames() { + return schema.getColumnNames(); + } + + @Override + public TypeInformation[] getFieldTypes() { + return schema.getTypes(); + } + + @Override + public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { + if (!Arrays.equals(getFieldNames(), fieldNames) || !Arrays.equals(getFieldTypes(), fieldTypes)) { + throw new ValidationException("Reconfiguration with different fields is not allowed. " + + "Expected: " + Arrays.toString(getFieldNames()) + " / " + Arrays.toString(getFieldTypes()) + ". " + + "But was: " + Arrays.toString(fieldNames) + " / " + Arrays.toString(fieldTypes)); + } + return copy( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + requestFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchUpsertTableSinkBase that = (ElasticsearchUpsertTableSinkBase) o; + return Objects.equals(isAppendOnly, that.isAppendOnly) && + Objects.equals(schema, that.schema) && + Objects.equals(hosts, that.hosts) && + Objects.equals(index, that.index) && + Objects.equals(docType, that.docType) && + Objects.equals(keyDelimiter, that.keyDelimiter) && + Objects.equals(keyNullLiteral, that.keyNullLiteral) && + Objects.equals(serializationSchema, that.serializationSchema) && + Objects.equals(contentType, that.contentType) && + Objects.equals(failureHandler, that.failureHandler) && + Objects.equals(sinkOptions, that.sinkOptions); + } + + @Override + public int hashCode() { + return Objects.hash( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // For version-specific implementations + // -------------------------------------------------------------------------------------------- + + protected abstract ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory); + + protected abstract SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertFunction); + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + /** + * Validate the types that are used for conversion to string. + */ + private void validateKeyTypes(int[] keyFieldIndices) { + final TypeInformation[] types = getFieldTypes(); + for (int keyFieldIndex : keyFieldIndices) { + final TypeInformation type = types[keyFieldIndex]; + if (!TypeCheckUtils.isSimpleStringRepresentation(type)) { + throw new ValidationException( + "Only simple types that can be safely converted into a string representation " + + "can be used as keys. But was: " + type); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** + * Keys for optional parameterization of the sink. + */ + public enum SinkOption { + DISABLE_FLUSH_ON_CHECKPOINT, + BULK_FLUSH_MAX_ACTIONS, + BULK_FLUSH_MAX_SIZE, + BULK_FLUSH_INTERVAL, + BULK_FLUSH_BACKOFF_ENABLED, + BULK_FLUSH_BACKOFF_TYPE, + BULK_FLUSH_BACKOFF_RETRIES, + BULK_FLUSH_BACKOFF_DELAY, + REST_MAX_RETRY_TIMEOUT, + REST_PATH_PREFIX + } + + /** + * Entity for describing a host of Elasticsearch. + */ + public static class Host { + public final String hostname; + public final int port; + public final String protocol; + + public Host(String hostname, int port, String protocol) { + this.hostname = hostname; + this.port = port; + this.protocol = protocol; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Host host = (Host) o; + return port == host.port && + Objects.equals(hostname, host.hostname) && + Objects.equals(protocol, host.protocol); + } + + @Override + public int hashCode() { + return Objects.hash( + hostname, + port, + protocol); + } + } + + /** + * For version-agnostic creating of {@link ActionRequest}s. + */ + public interface RequestFactory extends Serializable { + + /** + * Creates an update request to be added to a {@link RequestIndexer}. + */ + UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. + */ + IndexRequest createIndexRequest( + String index, + String docType, + XContentType contentType, + byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. + */ + DeleteRequest createDeleteRequest( + String index, + String docType, + String key); + } + + /** + * Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. + */ + public static class ElasticsearchUpsertSinkFunction implements ElasticsearchSinkFunction> { + + private final String index; + private final String docType; + private final String keyDelimiter; + private final String keyNullLiteral; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final int[] keyFieldIndices; + + public ElasticsearchUpsertSinkFunction( + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + int[] keyFieldIndices) { + + this.index = Preconditions.checkNotNull(index); + this.docType = Preconditions.checkNotNull(docType); + this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.keyFieldIndices = Preconditions.checkNotNull(keyFieldIndices); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); + } + + @Override + public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + if (element.f0) { + processUpsert(element.f1, indexer); + } else { + processDelete(element.f1, indexer); + } + } + + private void processUpsert(Row row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + if (keyFieldIndices.length == 0) { + final IndexRequest indexRequest = requestFactory.createIndexRequest( + index, + docType, + contentType, + document); + indexer.add(indexRequest); + } else { + final String key = createKey(row); + final UpdateRequest updateRequest = requestFactory.createUpdateRequest( + index, + docType, + key, + contentType, + document); + indexer.add(updateRequest); + } + } + + private void processDelete(Row row, RequestIndexer indexer) { + final String key = createKey(row); + final DeleteRequest deleteRequest = requestFactory.createDeleteRequest( + index, + docType, + key); + indexer.add(deleteRequest); + } + + private String createKey(Row row) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < keyFieldIndices.length; i++) { + final int keyFieldIndex = keyFieldIndices[i]; + if (i > 0) { + builder.append(keyDelimiter); + } + final Object value = row.getField(keyFieldIndex); + if (value == null) { + builder.append(keyNullLiteral); + } else { + builder.append(value.toString()); + } + } + return builder.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchUpsertSinkFunction that = (ElasticsearchUpsertSinkFunction) o; + return Objects.equals(index, that.index) && + Objects.equals(docType, that.docType) && + Objects.equals(keyDelimiter, that.keyDelimiter) && + Objects.equals(keyNullLiteral, that.keyNullLiteral) && + Objects.equals(serializationSchema, that.serializationSchema) && + contentType == that.contentType && + Objects.equals(requestFactory, that.requestFactory) && + Arrays.equals(keyFieldIndices, that.keyFieldIndices); + } + + @Override + public int hashCode() { + int result = Objects.hash( + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + requestFactory); + result = 31 * result + Arrays.hashCode(keyFieldIndices); + return result; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java new file mode 100644 index 00000000..79946deb --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -0,0 +1,296 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.ElasticsearchValidator; +import org.apache.flink.table.descriptors.SchemaValidator; +import org.apache.flink.table.descriptors.StreamTableDescriptorValidator; +import org.apache.flink.table.factories.SerializationSchemaFactory; +import org.apache.flink.table.factories.StreamTableSinkFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.sinks.StreamTableSink; +import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.types.Row; +import org.apache.flink.util.InstantiationUtil; + +import org.elasticsearch.common.xcontent.XContentType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_DELAY; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_INTERVAL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_ACTIONS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_SIZE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_PATH_PREFIX; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_DOCUMENT_TYPE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_CLASS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FLUSH_ON_CHECKPOINT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_HOSTNAME; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PORT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PROTOCOL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_INDEX; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; +import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; +import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; + +/** + * Version-agnostic table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch. + */ +@Internal +public abstract class ElasticsearchUpsertTableSinkFactoryBase implements StreamTableSinkFactory> { + + private static final String SUPPORTED_FORMAT_TYPE = "json"; + private static final XContentType SUPPORTED_CONTENT_TYPE = XContentType.JSON; + private static final String DEFAULT_KEY_DELIMITER = "_"; + private static final String DEFAULT_KEY_NULL_LITERAL = "null"; + private static final String DEFAULT_FAILURE_HANDLER = CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_ELASTICSEARCH); + context.put(CONNECTOR_VERSION(), elasticsearchVersion()); + context.put(CONNECTOR_PROPERTY_VERSION(), "1"); + return context; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + + // streaming properties + properties.add(UPDATE_MODE()); + + // Elasticsearch + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PORT); + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PROTOCOL); + properties.add(CONNECTOR_INDEX); + properties.add(CONNECTOR_DOCUMENT_TYPE); + properties.add(CONNECTOR_KEY_DELIMITER); + properties.add(CONNECTOR_KEY_NULL_LITERAL); + properties.add(CONNECTOR_FAILURE_HANDLER); + properties.add(CONNECTOR_FAILURE_HANDLER_CLASS); + properties.add(CONNECTOR_FLUSH_ON_CHECKPOINT); + properties.add(CONNECTOR_BULK_FLUSH_MAX_ACTIONS); + properties.add(CONNECTOR_BULK_FLUSH_MAX_SIZE); + properties.add(CONNECTOR_BULK_FLUSH_INTERVAL); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY); + properties.add(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT); + properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); + + // schema + properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); + properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + + // format wildcard + properties.add(FORMAT() + ".*"); + + return properties; + } + + @Override + public StreamTableSink> createStreamTableSink(Map properties) { + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); + + return createElasticsearchUpsertTableSink( + descriptorProperties.isValue(UPDATE_MODE(), UPDATE_MODE_VALUE_APPEND()), + descriptorProperties.getTableSchema(SCHEMA()), + getHosts(descriptorProperties), + descriptorProperties.getString(CONNECTOR_INDEX), + descriptorProperties.getString(CONNECTOR_DOCUMENT_TYPE), + descriptorProperties.getOptionalString(CONNECTOR_KEY_DELIMITER).orElse(DEFAULT_KEY_DELIMITER), + descriptorProperties.getOptionalString(CONNECTOR_KEY_NULL_LITERAL).orElse(DEFAULT_KEY_NULL_LITERAL), + getSerializationSchema(properties), + SUPPORTED_CONTENT_TYPE, + getFailureHandler(descriptorProperties), + getSinkOptions(descriptorProperties)); + } + + // -------------------------------------------------------------------------------------------- + // For version-specific factories + // -------------------------------------------------------------------------------------------- + + protected abstract String elasticsearchVersion(); + + protected abstract ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions); + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + private DescriptorProperties getValidatedProperties(Map properties) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + new StreamTableDescriptorValidator(true, false, true).validate(descriptorProperties); + new SchemaValidator(true, false, false).validate(descriptorProperties); + new ElasticsearchValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + private List getHosts(DescriptorProperties descriptorProperties) { + final List> hosts = descriptorProperties.getFixedIndexedProperties( + CONNECTOR_HOSTS, + Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL)); + return hosts.stream() + .map(host -> new Host( + descriptorProperties.getString(host.get(CONNECTOR_HOSTS_HOSTNAME)), + descriptorProperties.getInt(host.get(CONNECTOR_HOSTS_PORT)), + descriptorProperties.getString(host.get(CONNECTOR_HOSTS_PROTOCOL)))) + .collect(Collectors.toList()); + } + + private SerializationSchema getSerializationSchema(Map properties) { + final String formatType = properties.get(FORMAT_TYPE()); + // we could have added this check to the table factory context + // but this approach allows to throw more helpful error messages + // if the supported format has not been added + if (formatType == null || !formatType.equals(SUPPORTED_FORMAT_TYPE)) { + throw new ValidationException( + "The Elasticsearch sink requires a '" + SUPPORTED_FORMAT_TYPE + "' format."); + } + + @SuppressWarnings("unchecked") + final SerializationSchemaFactory formatFactory = TableFactoryService.find( + SerializationSchemaFactory.class, + properties, + this.getClass().getClassLoader()); + return formatFactory.createSerializationSchema(properties); + } + + private ActionRequestFailureHandler getFailureHandler(DescriptorProperties descriptorProperties) { + final String failureHandler = descriptorProperties + .getOptionalString(CONNECTOR_FAILURE_HANDLER) + .orElse(DEFAULT_FAILURE_HANDLER); + switch (failureHandler) { + case CONNECTOR_FAILURE_HANDLER_VALUE_FAIL: + return new NoOpFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE: + return new IgnoringFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_RETRY: + return new RetryRejectedExecutionFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM: + final Class clazz = descriptorProperties + .getClass(CONNECTOR_FAILURE_HANDLER_CLASS, ActionRequestFailureHandler.class); + return InstantiationUtil.instantiate(clazz); + default: + throw new IllegalArgumentException("Unknown failure handler."); + } + } + + private Map getSinkOptions(DescriptorProperties descriptorProperties) { + final Map options = new HashMap<>(); + + descriptorProperties.getOptionalBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT) + .ifPresent(v -> options.put(SinkOption.DISABLE_FLUSH_ON_CHECKPOINT, String.valueOf(!v))); + + mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_MAX_ACTIONS, SinkOption.BULK_FLUSH_MAX_ACTIONS); + mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_MAX_SIZE, SinkOption.BULK_FLUSH_MAX_SIZE); + mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_INTERVAL, SinkOption.BULK_FLUSH_INTERVAL); + + descriptorProperties.getOptionalString(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE) + .ifPresent(v -> { + options.put( + SinkOption.BULK_FLUSH_BACKOFF_ENABLED, + String.valueOf(!v.equals(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED))); + switch (v) { + case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT: + options.put( + SinkOption.BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchSinkBase.FlushBackoffType.CONSTANT.toString()); + break; + case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL: + options.put( + SinkOption.BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL.toString()); + break; + default: + throw new IllegalArgumentException("Unknown backoff type."); + } + }); + + mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, SinkOption.BULK_FLUSH_BACKOFF_RETRIES); + mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, SinkOption.BULK_FLUSH_BACKOFF_DELAY); + mapSinkOption(descriptorProperties, options, CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, SinkOption.REST_MAX_RETRY_TIMEOUT); + mapSinkOption(descriptorProperties, options, CONNECTOR_CONNECTION_PATH_PREFIX, SinkOption.REST_PATH_PREFIX); + + return options; + } + + private void mapSinkOption( + DescriptorProperties descriptorProperties, + Map options, + String fromKey, + SinkOption toKey) { + descriptorProperties.getOptionalString(fromKey).ifPresent(v -> options.put(toKey, v)); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java new file mode 100644 index 00000000..a3644b45 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java @@ -0,0 +1,39 @@ +/* + * 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.connectors.elasticsearch.util; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; + +/** + * Ignores all kinds of failures and drops the affected {@link ActionRequest}. + */ +@Internal +public class IgnoringFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 1662846593501L; + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // ignore failure + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java new file mode 100644 index 00000000..f306e192 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -0,0 +1,315 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.util.Preconditions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_DELAY; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_INTERVAL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_ACTIONS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_SIZE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_PATH_PREFIX; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_DOCUMENT_TYPE; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_CLASS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FLUSH_ON_CHECKPOINT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_HOSTNAME; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PORT; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PROTOCOL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_INDEX; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; + +/** + * Connector descriptor for the Elasticsearch search engine. + */ +public class Elasticsearch extends ConnectorDescriptor { + + private DescriptorProperties internalProperties = new DescriptorProperties(true); + private List hosts = new ArrayList<>(); + + /** + * Connector descriptor for the Elasticsearch search engine. + */ + public Elasticsearch() { + super(CONNECTOR_TYPE_VALUE_ELASTICSEARCH, 1, true); + } + + /** + * Sets the Elasticsearch version to be used. Required. + * + * @param version Elasticsearch version. E.g., "6". + */ + public Elasticsearch version(String version) { + internalProperties.putString(CONNECTOR_VERSION(), version); + return this; + } + + /** + * Adds an Elasticsearch host to connect to. Required. + * + *

Multiple hosts can be declared by calling this method multiple times. + * + * @param hostname connection hostname + * @param port connection port + * @param protocol connection protocol; e.g. "http" + */ + public Elasticsearch host(String hostname, int port, String protocol) { + final Host host = + new Host( + Preconditions.checkNotNull(hostname), + port, + Preconditions.checkNotNull(protocol)); + hosts.add(host); + return this; + } + + /** + * Declares the Elasticsearch index for every record. Required. + * + * @param index Elasticsearch index + */ + public Elasticsearch index(String index) { + internalProperties.putString(CONNECTOR_INDEX, index); + return this; + } + + /** + * Declares the Elasticsearch document type for every record. Required. + * + * @param documentType Elasticsearch document type + */ + public Elasticsearch documentType(String documentType) { + internalProperties.putString(CONNECTOR_DOCUMENT_TYPE, documentType); + return this; + } + + /** + * Sets a custom key delimiter in case the Elasticsearch ID needs to be constructed from + * multiple fields. Optional. + * + * @param keyDelimiter key delimiter; e.g., "$" would result in IDs "KEY1$KEY2$KEY3" + */ + public Elasticsearch keyDelimiter(String keyDelimiter) { + internalProperties.putString(CONNECTOR_KEY_DELIMITER, keyDelimiter); + return this; + } + + /** + * Sets a custom representation for null fields in keys. Optional. + * + * @param keyNullLiteral key null literal string; e.g. "N/A" would result in IDs "KEY1_N/A_KEY3" + */ + public Elasticsearch keyNullLiteral(String keyNullLiteral) { + internalProperties.putString(CONNECTOR_KEY_NULL_LITERAL, keyNullLiteral); + return this; + } + + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy throws an exception if a request fails and thus causes a job failure. + */ + public Elasticsearch failureHandlerFail() { + internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL); + return this; + } + + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy ignores failures and drops the request. + */ + public Elasticsearch failureHandlerIgnore() { + internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE); + return this; + } + + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy re-adds requests that have failed due to queue capacity saturation. + */ + public Elasticsearch failureHandlerRetryRejected() { + internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY); + return this; + } + + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy allows for custom failure handling using a {@link ActionRequestFailureHandler}. + */ + public Elasticsearch failureHandlerCustom(Class failureHandlerClass) { + internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM); + internalProperties.putClass(CONNECTOR_FAILURE_HANDLER_CLASS, failureHandlerClass); + return this; + } + + /** + * Disables flushing on checkpoint. When disabled, a sink will not wait for all pending action + * requests to be acknowledged by Elasticsearch on checkpoints. + * + *

Note: If flushing on checkpoint is disabled, a Elasticsearch sink does NOT + * provide any strong guarantees for at-least-once delivery of action requests. + */ + public Elasticsearch disableFlushOnCheckpoint() { + internalProperties.putBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, false); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum number of actions to buffer for each bulk request. + * + * @param maxActions the maximum number of actions to buffer per bulk request. + */ + public Elasticsearch bulkFlushMaxActions(int maxActions) { + internalProperties.putInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, maxActions); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum size of buffered actions per bulk request (using the syntax of {@link MemorySize}). + */ + public Elasticsearch bulkFlushMaxSize(String maxSize) { + internalProperties.putMemorySize(CONNECTOR_BULK_FLUSH_MAX_SIZE, MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the bulk flush interval (in milliseconds). + * + * @param interval bulk flush interval (in milliseconds). + */ + public Elasticsearch bulkFlushInterval(long interval) { + internalProperties.putLong(CONNECTOR_BULK_FLUSH_INTERVAL, interval); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets a constant backoff type to use when flushing bulk requests. + */ + public Elasticsearch bulkFlushBackoffConstant() { + internalProperties.putString( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets an exponential backoff type to use when flushing bulk requests. + */ + public Elasticsearch bulkFlushBackoffExponential() { + internalProperties.putString( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} or + * {@link #bulkFlushBackoffExponential()}). + * + * @param maxRetries the maximum number of retries. + */ + public Elasticsearch bulkFlushBackoffMaxRetries(int maxRetries) { + internalProperties.putInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, maxRetries); + return this; + } + + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the amount of delay between each backoff attempt when flushing bulk requests (in milliseconds). + * + *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} or + * {@link #bulkFlushBackoffExponential()}). + * + * @param delay delay between each backoff attempt (in milliseconds). + */ + public Elasticsearch bulkFlushBackoffDelay(long delay) { + internalProperties.putLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, delay); + return this; + } + + /** + * Sets connection properties to be used during REST communication to Elasticsearch. + * + *

Sets the maximum timeout (in milliseconds) in case of multiple retries of the same request. + * + * @param maxRetryTimeout maximum timeout (in milliseconds) + */ + public Elasticsearch connectionMaxRetryTimeout(int maxRetryTimeout) { + internalProperties.putInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, maxRetryTimeout); + return this; + } + + /** + * Sets connection properties to be used during REST communication to Elasticsearch. + * + *

Adds a path prefix to every REST communication. + * + * @param pathPrefix prefix string to be added to every REST communication + */ + public Elasticsearch connectionPathPrefix(String pathPrefix) { + internalProperties.putString(CONNECTOR_CONNECTION_PATH_PREFIX, pathPrefix); + return this; + } + + @Override + public void addConnectorProperties(DescriptorProperties properties) { + properties.putProperties(internalProperties.asMap()); + + final List> hostValues = hosts.stream() + .map(host -> Arrays.asList(host.hostname, String.valueOf(host.port), host.protocol)) + .collect(Collectors.toList()); + properties.putIndexedFixedProperties( + CONNECTOR_HOSTS, + Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL), + hostValues); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java new file mode 100644 index 00000000..9e04f2f7 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -0,0 +1,129 @@ +/* + * 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.table.descriptors; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Consumer; + +/** + * The validator for {@link Elasticsearch}. + */ +public class ElasticsearchValidator extends ConnectorDescriptorValidator { + + public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; + public static final String CONNECTOR_VERSION_VALUE_6 = "6"; + public static final String CONNECTOR_HOSTS = "connector.hosts"; + public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; + public static final String CONNECTOR_HOSTS_PORT = "port"; + public static final String CONNECTOR_HOSTS_PROTOCOL = "protocol"; + public static final String CONNECTOR_INDEX = "connector.index"; + public static final String CONNECTOR_DOCUMENT_TYPE = "connector.document-type"; + public static final String CONNECTOR_KEY_DELIMITER = "connector.key-delimiter"; + public static final String CONNECTOR_KEY_NULL_LITERAL = "connector.key-null-literal"; + public static final String CONNECTOR_FAILURE_HANDLER = "connector.failure-handler"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_FAIL = "fail"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE = "ignore"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_RETRY = "retry-rejected"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM = "custom"; + public static final String CONNECTOR_FAILURE_HANDLER_CLASS = "connector.failure-handler-class"; + public static final String CONNECTOR_FLUSH_ON_CHECKPOINT = "connector.flush-on-checkpoint"; + public static final String CONNECTOR_BULK_FLUSH_MAX_ACTIONS = "connector.bulk-flush.max-actions"; + public static final String CONNECTOR_BULK_FLUSH_MAX_SIZE = "connector.bulk-flush.max-size"; + public static final String CONNECTOR_BULK_FLUSH_INTERVAL = "connector.bulk-flush.interval"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE = "connector.bulk-flush.backoff.type"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED = "disabled"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT = "constant"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL = "exponential"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES = "connector.bulk-flush.backoff.max-retries"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_DELAY = "connector.bulk-flush.backoff.delay"; + public static final String CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT = "connector.connection-max-retry-timeout"; + public static final String CONNECTOR_CONNECTION_PATH_PREFIX = "connector.connection-path-prefix"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateValue(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); + validateVersion(properties); + validateHosts(properties); + validateGeneralProperties(properties); + validateFailureHandler(properties); + validateBulkFlush(properties); + validateConnectionProperties(properties); + } + + private void validateVersion(DescriptorProperties properties) { + properties.validateEnumValues( + CONNECTOR_VERSION(), + false, + Collections.singletonList(CONNECTOR_VERSION_VALUE_6)); + } + + private void validateHosts(DescriptorProperties properties) { + final Map> hostsValidators = new HashMap<>(); + hostsValidators.put( + CONNECTOR_HOSTS_HOSTNAME, + (prefix) -> properties.validateString(prefix + CONNECTOR_HOSTS_HOSTNAME, false, 1)); + hostsValidators.put( + CONNECTOR_HOSTS_PORT, + (prefix) -> properties.validateInt(prefix + CONNECTOR_HOSTS_PORT, false, 0, 65535)); + hostsValidators.put(CONNECTOR_HOSTS_PROTOCOL, + (prefix) -> properties.validateString(prefix + CONNECTOR_HOSTS_PROTOCOL, false, 1)); + properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); + } + + private void validateGeneralProperties(DescriptorProperties properties) { + properties.validateString(CONNECTOR_INDEX, false, 1); + properties.validateString(CONNECTOR_DOCUMENT_TYPE, false, 1); + properties.validateString(CONNECTOR_KEY_DELIMITER, true); + properties.validateString(CONNECTOR_KEY_NULL_LITERAL, true); + } + + private void validateFailureHandler(DescriptorProperties properties) { + final Map> failureHandlerValidators = new HashMap<>(); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_FAIL, properties.noValidation()); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE, properties.noValidation()); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_RETRY, properties.noValidation()); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM, + prefix -> properties.validateString(CONNECTOR_FAILURE_HANDLER_CLASS, false, 1)); + properties.validateEnum(CONNECTOR_FAILURE_HANDLER, true, failureHandlerValidators); + } + + private void validateBulkFlush(DescriptorProperties properties) { + properties.validateBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, true); + properties.validateInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, true, 1); + properties.validateMemorySize(CONNECTOR_BULK_FLUSH_MAX_SIZE, true, 1024 * 1024); // only allow MB precision + properties.validateLong(CONNECTOR_BULK_FLUSH_INTERVAL, true, 0); + properties.validateEnumValues(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + true, + Arrays.asList( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED, + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT, + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL)); + properties.validateInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, true, 1); + properties.validateLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, true, 0); + } + + private void validateConnectionProperties(DescriptorProperties properties) { + properties.validateInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, true, 1); + properties.validateString(CONNECTOR_CONNECTION_PATH_PREFIX, true); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java new file mode 100644 index 00000000..9376bfb4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -0,0 +1,189 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.Types; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.apache.flink.table.descriptors.Elasticsearch; +import org.apache.flink.table.descriptors.Json; +import org.apache.flink.table.descriptors.Schema; +import org.apache.flink.table.descriptors.TestTableDescriptor; +import org.apache.flink.table.factories.StreamTableSinkFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import org.apache.flink.util.TestLogger; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.common.xcontent.XContentType; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** + * Version-agnostic test base for {@link ElasticsearchUpsertTableSinkFactoryBase}. + */ +public abstract class ElasticsearchUpsertTableSinkFactoryTestBase extends TestLogger { + + protected static final String HOSTNAME = "host1"; + protected static final int PORT = 1234; + protected static final String SCHEMA = "https"; + protected static final String INDEX = "MyIndex"; + protected static final String DOC_TYPE = "MyType"; + protected static final String KEY_DELIMITER = "#"; + protected static final String KEY_NULL_LITERAL = ""; + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + @Test + public void testTableSink() { + // prepare parameters for Elasticsearch table sink + + final TableSchema schema = createTestSchema(); + + final ElasticsearchUpsertTableSinkBase expectedSink = getExpectedTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + // construct table sink using descriptors and table sink factory + + final TestTableDescriptor testDesc = new TestTableDescriptor( + new Elasticsearch() + .version(getElasticsearchVersion()) + .host(HOSTNAME, PORT, SCHEMA) + .index(INDEX) + .documentType(DOC_TYPE) + .keyDelimiter(KEY_DELIMITER) + .keyNullLiteral(KEY_NULL_LITERAL) + .bulkFlushBackoffExponential() + .bulkFlushBackoffDelay(123L) + .bulkFlushBackoffMaxRetries(3) + .bulkFlushInterval(100L) + .bulkFlushMaxActions(1000) + .bulkFlushMaxSize("1 MB") + .failureHandlerCustom(DummyFailureHandler.class) + .connectionMaxRetryTimeout(100) + .connectionPathPrefix("/myapp")) + .withFormat( + new Json() + .deriveSchema()) + .withSchema( + new Schema() + .field(FIELD_KEY, Types.LONG()) + .field(FIELD_FRUIT_NAME, Types.STRING()) + .field(FIELD_COUNT, Types.DECIMAL()) + .field(FIELD_TS, Types.SQL_TIMESTAMP())) + .inUpsertMode(); + + final Map propertiesMap = DescriptorProperties.toJavaMap(testDesc); + final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, propertiesMap) + .createStreamTableSink(propertiesMap); + + assertEquals(expectedSink, actualSink); + } + + protected TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, Types.LONG()) + .field(FIELD_FRUIT_NAME, Types.STRING()) + .field(FIELD_COUNT, Types.DECIMAL()) + .field(FIELD_TS, Types.SQL_TIMESTAMP()) + .build(); + } + + protected Map createTestSinkOptions() { + final Map sinkOptions = new HashMap<>(); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_ENABLED, "true"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_TYPE, "EXPONENTIAL"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_DELAY, "123"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_RETRIES, "3"); + sinkOptions.put(SinkOption.BULK_FLUSH_INTERVAL, "100"); + sinkOptions.put(SinkOption.BULK_FLUSH_MAX_ACTIONS, "1000"); + sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1048576 bytes"); + sinkOptions.put(SinkOption.REST_MAX_RETRY_TIMEOUT, "100"); + sinkOptions.put(SinkOption.REST_PATH_PREFIX, "/myapp"); + return sinkOptions; + } + + // -------------------------------------------------------------------------------------------- + // For version-specific tests + // -------------------------------------------------------------------------------------------- + + protected abstract String getElasticsearchVersion(); + + protected abstract ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions); + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** + * Custom failure handler for testing. + */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return this == o || o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java new file mode 100644 index 00000000..c97aa0e6 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java @@ -0,0 +1,147 @@ +/* + * 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.table.descriptors; + +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.table.api.ValidationException; + +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Tests for the {@link Elasticsearch} descriptor. + */ +public class ElasticsearchTest extends DescriptorTestBase { + + @Test(expected = ValidationException.class) + public void testMissingIndex() { + removePropertyAndVerify(descriptors().get(0), "connector.index"); + } + + @Test(expected = ValidationException.class) + public void testInvalidFailureHandler() { + addPropertyAndVerify(descriptors().get(0), "connector.failure-handler", "invalid handler"); + } + + @Test(expected = ValidationException.class) + public void testInvalidMemorySize() { + addPropertyAndVerify(descriptors().get(1), "connector.bulk-flush.max-size", "12 bytes"); + } + + @Override + public List descriptors() { + final Descriptor minimumDesc = + new Elasticsearch() + .version("6") + .host("localhost", 1234, "http") + .index("MyIndex") + .documentType("MyType"); + + final Descriptor maximumDesc = + new Elasticsearch() + .version("6") + .host("host1", 1234, "https") + .host("host2", 1234, "https") + .index("MyIndex") + .documentType("MyType") + .keyDelimiter("#") + .keyNullLiteral("") + .bulkFlushBackoffExponential() + .bulkFlushBackoffDelay(123L) + .bulkFlushBackoffMaxRetries(3) + .bulkFlushInterval(100L) + .bulkFlushMaxActions(1000) + .bulkFlushMaxSize("12 MB") + .failureHandlerRetryRejected() + .connectionMaxRetryTimeout(100) + .connectionPathPrefix("/myapp"); + + final Descriptor customDesc = + new Elasticsearch() + .version("6") + .host("localhost", 1234, "http") + .index("MyIndex") + .documentType("MyType") + .disableFlushOnCheckpoint() + .failureHandlerCustom(NoOpFailureHandler.class); + + return Arrays.asList(minimumDesc, maximumDesc, customDesc); + } + + @Override + public List> properties() { + final Map minimumDesc = new HashMap<>(); + minimumDesc.put("connector.property-version", "1"); + minimumDesc.put("connector.type", "elasticsearch"); + minimumDesc.put("connector.version", "6"); + minimumDesc.put("connector.hosts.0.hostname", "localhost"); + minimumDesc.put("connector.hosts.0.port", "1234"); + minimumDesc.put("connector.hosts.0.protocol", "http"); + minimumDesc.put("connector.index", "MyIndex"); + minimumDesc.put("connector.document-type", "MyType"); + + final Map maximumDesc = new HashMap<>(); + maximumDesc.put("connector.property-version", "1"); + maximumDesc.put("connector.type", "elasticsearch"); + maximumDesc.put("connector.version", "6"); + maximumDesc.put("connector.hosts.0.hostname", "host1"); + maximumDesc.put("connector.hosts.0.port", "1234"); + maximumDesc.put("connector.hosts.0.protocol", "https"); + maximumDesc.put("connector.hosts.1.hostname", "host2"); + maximumDesc.put("connector.hosts.1.port", "1234"); + maximumDesc.put("connector.hosts.1.protocol", "https"); + maximumDesc.put("connector.index", "MyIndex"); + maximumDesc.put("connector.document-type", "MyType"); + maximumDesc.put("connector.key-delimiter", "#"); + maximumDesc.put("connector.key-null-literal", ""); + maximumDesc.put("connector.bulk-flush.backoff.type", "exponential"); + maximumDesc.put("connector.bulk-flush.backoff.delay", "123"); + maximumDesc.put("connector.bulk-flush.backoff.max-retries", "3"); + maximumDesc.put("connector.bulk-flush.interval", "100"); + maximumDesc.put("connector.bulk-flush.max-actions", "1000"); + maximumDesc.put("connector.bulk-flush.max-size", "12582912 bytes"); + maximumDesc.put("connector.failure-handler", "retry-rejected"); + maximumDesc.put("connector.connection-max-retry-timeout", "100"); + maximumDesc.put("connector.connection-path-prefix", "/myapp"); + + final Map customDesc = new HashMap<>(); + customDesc.put("connector.property-version", "1"); + customDesc.put("connector.type", "elasticsearch"); + customDesc.put("connector.version", "6"); + customDesc.put("connector.hosts.0.hostname", "localhost"); + customDesc.put("connector.hosts.0.port", "1234"); + customDesc.put("connector.hosts.0.protocol", "http"); + customDesc.put("connector.index", "MyIndex"); + customDesc.put("connector.document-type", "MyType"); + customDesc.put("connector.flush-on-checkpoint", "false"); + customDesc.put("connector.failure-handler", "custom"); + customDesc.put("connector.failure-handler-class", NoOpFailureHandler.class.getName()); + + return Arrays.asList(minimumDesc, maximumDesc, customDesc); + } + + @Override + public DescriptorValidator validator() { + return new ElasticsearchValidator(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index ef06d805..427ad5ce 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -84,6 +84,16 @@ under the License. 2.9.1 + + + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + provided + + true + + @@ -140,20 +150,119 @@ under the License. org.apache.logging.log4j - log4j-api + log4j-core 2.9.1 test + - org.apache.logging.log4j - log4j-core - 2.9.1 + org.apache.flink + flink-table_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} test + + + + sql-jars + + + !skipSqlJars + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + sql-jar + + + *:* + + + + com/carrotsearch/** + com/sun/** + com/tdunning/** + config/** + forbidden/** + joptsimple/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.elasticsearch.** + META-INF/versions/** + modules.txt + mozilla/** + org/HdrHistogram/** + org/joda/** + org/tartarus/** + org/yaml/** + plugins.txt + + + + + + + org.apache.commons + org.apache.flink.elasticsearch6.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch6.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch6.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch6.shaded.org.elasticsearch + + + org.apache.logging + org.apache.flink.elasticsearch6.shaded.org.apache.logging + + + com.fasterxml.jackson + org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson + + + + + + + + + + + + + + + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + org.apache.flink diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index 79946deb..63e9b342 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -102,9 +102,9 @@ public abstract class ElasticsearchUpsertTableSinkFactoryBase implements StreamT @Override public Map requiredContext() { final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_ELASTICSEARCH); - context.put(CONNECTOR_VERSION(), elasticsearchVersion()); - context.put(CONNECTOR_PROPERTY_VERSION(), "1"); + context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH); + context.put(CONNECTOR_VERSION, elasticsearchVersion()); + context.put(CONNECTOR_PROPERTY_VERSION, "1"); return context; } @@ -140,7 +140,7 @@ public List supportedProperties() { properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); // format wildcard - properties.add(FORMAT() + ".*"); + properties.add(FORMAT + ".*"); return properties; } @@ -210,7 +210,7 @@ private List getHosts(DescriptorProperties descriptorProperties) { } private SerializationSchema getSerializationSchema(Map properties) { - final String formatType = properties.get(FORMAT_TYPE()); + final String formatType = properties.get(FORMAT_TYPE); // we could have added this check to the table factory context // but this approach allows to throw more helpful error messages // if the supported format has not been added diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java index f306e192..fa07821e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.stream.Collectors; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; @@ -71,7 +72,7 @@ public Elasticsearch() { * @param version Elasticsearch version. E.g., "6". */ public Elasticsearch version(String version) { - internalProperties.putString(CONNECTOR_VERSION(), version); + internalProperties.putString(CONNECTOR_VERSION, version); return this; } @@ -301,8 +302,9 @@ public Elasticsearch connectionPathPrefix(String pathPrefix) { } @Override - public void addConnectorProperties(DescriptorProperties properties) { - properties.putProperties(internalProperties.asMap()); + protected Map toConnectorProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(internalProperties); final List> hostValues = hosts.stream() .map(host -> Arrays.asList(host.hostname, String.valueOf(host.port), host.protocol)) @@ -311,5 +313,7 @@ public void addConnectorProperties(DescriptorProperties properties) { CONNECTOR_HOSTS, Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL), hostValues); + + return properties.asMap(); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java index 1858ee19..9d1a926e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -18,6 +18,8 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.annotation.Internal; + import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -29,6 +31,7 @@ /** * The validator for {@link Elasticsearch}. */ +@Internal public class ElasticsearchValidator extends ConnectorDescriptorValidator { public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; @@ -63,7 +66,7 @@ public class ElasticsearchValidator extends ConnectorDescriptorValidator { @Override public void validate(DescriptorProperties properties) { super.validate(properties); - properties.validateValue(CONNECTOR_TYPE(), CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); + properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); validateVersion(properties); validateHosts(properties); validateGeneralProperties(properties); @@ -74,7 +77,7 @@ public void validate(DescriptorProperties properties) { private void validateVersion(DescriptorProperties properties) { properties.validateEnumValues( - CONNECTOR_VERSION(), + CONNECTOR_VERSION, false, Collections.singletonList(CONNECTOR_VERSION_VALUE_6)); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index 88aee43f..0cc282c0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -31,7 +31,6 @@ import org.apache.flink.table.factories.StreamTableSinkFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.util.JavaScalaConversionUtil; import org.apache.flink.types.Row; import org.apache.flink.util.TestLogger; @@ -113,7 +112,7 @@ public void testTableSink() { .field(FIELD_TS, Types.SQL_TIMESTAMP())) .inUpsertMode(); - final Map propertiesMap = JavaScalaConversionUtil.toJavaMap(testDesc); + final Map propertiesMap = testDesc.toProperties(); final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, propertiesMap) .createStreamTableSink(propertiesMap); From c397f1bdfff1b94e9c34d617e17df165f7387ec0 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Thu, 25 Oct 2018 17:43:09 +0200 Subject: [PATCH 030/207] [FLINK-10681] Harden ElasticsearchSinkITCase against wrong JNA library Set the system property jna.nosys=true to avoid ElasticsearchSinkITCase test failures due a wrong JNA version. --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 427ad5ce..d1eb3d8b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -278,6 +278,9 @@ under the License. maven-surefire-plugin 2.12.2 + + true + org.apache.logging.log4j:log4j-to-slf4j From b2f1be35228c5877fae2c782aa17e9829915ac98 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 3 Nov 2018 12:08:28 +0100 Subject: [PATCH 031/207] Update version to 1.8-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index b7b79406..b70c0bc1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.7-SNAPSHOT + 1.8-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index d1eb3d8b..40b9ab6e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.7-SNAPSHOT + 1.8-SNAPSHOT .. From 15d14b9a5006808b1ff4972bd8e9354fe35c429b Mon Sep 17 00:00:00 2001 From: zentol Date: Wed, 5 Dec 2018 15:46:05 +0100 Subject: [PATCH 032/207] [FLINK-11080][ES] Rework shade-plugin filters --- .../flink-connector-elasticsearch6/pom.xml | 58 ++++++++++++++----- 1 file changed, 43 insertions(+), 15 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 40b9ab6e..57d10795 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -197,29 +197,57 @@ under the License. true sql-jar + + + *:* + + + + com.carrotsearch:hppc + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + - *:* - - + org.elasticsearch:elasticsearch - com/carrotsearch/** - com/sun/** - com/tdunning/** config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + forbidden/** - joptsimple/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** META-INF/services/com.fasterxml.** META-INF/services/org.apache.lucene.** META-INF/services/org.elasticsearch.** - META-INF/versions/** - modules.txt - mozilla/** - org/HdrHistogram/** - org/joda/** - org/tartarus/** - org/yaml/** - plugins.txt From 4556937f4af9a0d3ab3fb5445f18879dc525b872 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 18 Dec 2018 13:08:58 +0100 Subject: [PATCH 033/207] [FLINK-11026][ES6] Rework creation of fat sql-client jar --- .../flink-connector-elasticsearch6/pom.xml | 117 ------------- .../pom.xml | 155 ++++++++++++++++++ 2 files changed, 155 insertions(+), 117 deletions(-) create mode 100644 flink-connectors/flink-sql-connector-elasticsearch6/pom.xml diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 57d10795..4c79e898 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -174,123 +174,6 @@ under the License. - - - - sql-jars - - - !skipSqlJars - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - true - sql-jar - - - *:* - - - - com.carrotsearch:hppc - com.tdunning:t-digest - joda-time:joda-time - net.sf.jopt-simple:jopt-simple - org.elasticsearch:jna - org.hdrhistogram:HdrHistogram - org.yaml:snakeyaml - - - - - org.elasticsearch:elasticsearch - - config/** - modules.txt - plugins.txt - org/joda/** - - - - org.elasticsearch.client:elasticsearch-rest-high-level-client - - forbidden/** - - - - org.apache.httpcomponents:httpclient - - mozilla/** - - - - org.apache.lucene:lucene-analyzers-common - - org/tartarus/** - - - - *:* - - - META-INF/versions/** - META-INF/services/com.fasterxml.** - META-INF/services/org.apache.lucene.** - META-INF/services/org.elasticsearch.** - - - - - - - org.apache.commons - org.apache.flink.elasticsearch6.shaded.org.apache.commons - - - org.apache.http - org.apache.flink.elasticsearch6.shaded.org.apache.http - - - org.apache.lucene - org.apache.flink.elasticsearch6.shaded.org.apache.lucene - - - org.elasticsearch - org.apache.flink.elasticsearch6.shaded.org.elasticsearch - - - org.apache.logging - org.apache.flink.elasticsearch6.shaded.org.apache.logging - - - com.fasterxml.jackson - org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson - - - - - - - - - - - - - - - + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.8-SNAPSHOT + .. + + + flink-sql-connector-elasticsearch6_${scala.binary.version} + flink-sql-connector-elasticsearch6 + + jar + + + + 6.3.1 + + + + + org.apache.flink + flink-connector-elasticsearch6_${scala.binary.version} + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + com.carrotsearch:hppc + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + + + + + org.elasticsearch:elasticsearch + + config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + + forbidden/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.elasticsearch.** + + + + + + + org.apache.commons + org.apache.flink.elasticsearch6.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch6.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch6.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch6.shaded.org.elasticsearch + + + org.apache.logging + org.apache.flink.elasticsearch6.shaded.org.apache.logging + + + com.fasterxml.jackson + org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson + + + + + + + + + + From 3a10dd8c7fed968317b2fc5979e74e8271fbbf6b Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 3 Jan 2019 19:35:49 +0100 Subject: [PATCH 034/207] [FLINK-11023][ES] Add NOTICE file --- .../pom.xml | 1 + .../src/main/resources/META-INF/NOTICE | 45 +++++++++++++++++++ 2 files changed, 46 insertions(+) create mode 100644 flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 85b73a9c..74385661 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -115,6 +115,7 @@ under the License. META-INF/services/com.fasterxml.** META-INF/services/org.apache.lucene.** META-INF/services/org.elasticsearch.** + META-INF/LICENSE.txt diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..9880ee92 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -0,0 +1,45 @@ +flink-sql-connector-elasticsearch6 +Copyright 2014-2018 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.fasterxml.jackson.core:jackson-core:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- org.apache.httpcomponents:httpasyncclient:4.1.2 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpcore-nio:4.4.5 +- org.apache.logging.log4j:log4j-api:2.9.1 +- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 +- org.apache.lucene:lucene-analyzers-common:7.3.1 +- org.apache.lucene:lucene-backward-codecs:7.3.1 +- org.apache.lucene:lucene-core:7.3.1 +- org.apache.lucene:lucene-grouping:7.3.1 +- org.apache.lucene:lucene-highlighter:7.3.1 +- org.apache.lucene:lucene-join:7.3.1 +- org.apache.lucene:lucene-memory:7.3.1 +- org.apache.lucene:lucene-misc:7.3.1 +- org.apache.lucene:lucene-queries:7.3.1 +- org.apache.lucene:lucene-queryparser:7.3.1 +- org.apache.lucene:lucene-sandbox:7.3.1 +- org.apache.lucene:lucene-spatial:7.3.1 +- org.apache.lucene:lucene-spatial-extras:7.3.1 +- org.apache.lucene:lucene-spatial3d:7.3.1 +- org.apache.lucene:lucene-suggest:7.3.1 +- org.elasticsearch:elasticsearch:6.3.1 +- org.elasticsearch:elasticsearch-cli:6.3.1 +- org.elasticsearch:elasticsearch-core:6.3.1 +- org.elasticsearch:elasticsearch-secure-sm:6.3.1 +- org.elasticsearch:elasticsearch-x-content:6.3.1 +- org.elasticsearch.client:elasticsearch-rest-client:6.3.1 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:6.3.1 +- org.elasticsearch.plugin:aggs-matrix-stats-client:6.3.1 +- org.elasticsearch.plugin:parent-join-client:6.3.1 +- org.elasticsearch.plugin:rank-eval-client:6.3.1 From ff4e9fb34a08b46af7de18b2e261539f951b3550 Mon Sep 17 00:00:00 2001 From: zentol Date: Tue, 18 Dec 2018 13:39:33 +0100 Subject: [PATCH 035/207] [hotfix][ES6][SQL] Remove unused version property --- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 74385661..95432f44 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -35,11 +35,6 @@ under the License. jar - - - 6.3.1 - - org.apache.flink From 6a9a2891b11dd304b62a3914fee710ab81401f15 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 28 Jan 2019 07:48:59 +0100 Subject: [PATCH 036/207] [FLINK-11064] [table] Setup a new flink-table module structure This commit splits the flink-table module into multiple submodules in accordance with FLIP-32 (step 1). The new module structure looks as follows: flink-table-common ^ | flink-table-api-java <------- flink-table-api-scala ^ ^ | | flink-table-api-java-bridge flink-table-api-scala-bridge ^ | flink-table-planner The module structure assumes that the type system has been reworked such that only one table environment exists for both Java and Scala users. The module `flink-table-planner` contains the content of the old `flink-table` module. From there we can distribute ported classes to their final module without breaking backwards compatibility or force users to update their dependencies again. For example, if a user wants to implement a pure table program in Scala, `flink-table-api-scala` and `flink-table-planner` need to be added to the project. Until we support pure table programs, `flink-table-api-scala/java-bridge` and `flink-table-planner` need to be added to the project. This closes #7587. --- .../flink-connector-elasticsearch-base/pom.xml | 16 ++++++++++++---- .../flink-connector-elasticsearch6/pom.xml | 16 ++++++++++++---- 2 files changed, 24 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index b70c0bc1..ebd722ae 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -70,13 +70,21 @@ under the License. - + + org.apache.flink - flink-table_${scala.binary.version} + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.apache.flink + flink-table-planner_${scala.binary.version} ${project.version} provided - true @@ -108,7 +116,7 @@ under the License. org.apache.flink - flink-table_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test-jar test diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 4c79e898..b2a98865 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -84,13 +84,21 @@ under the License. 2.9.1 - + + org.apache.flink - flink-table_${scala.binary.version} + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.apache.flink + flink-table-planner_${scala.binary.version} ${project.version} provided - true @@ -158,7 +166,7 @@ under the License. org.apache.flink - flink-table_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test-jar test From 6a5b101fb2b33d4ba23d38ecb3b0bae7ab8ff193 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 1 Feb 2019 20:26:27 +0800 Subject: [PATCH 037/207] [FLINK-11450][table] Port and move TableSource and TableSink related classes to flink-table-common This closes #7626. --- .../elasticsearch/ElasticsearchUpsertTableSinkBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 8e0279fb..045d3a6c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -31,7 +31,7 @@ import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sinks.UpsertStreamTableSink; import org.apache.flink.table.typeutils.TypeCheckUtils; -import org.apache.flink.table.util.TableConnectorUtil; +import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; @@ -185,7 +185,7 @@ public void emitDataStream(DataStream> dataStream) { sinkOptions, upsertFunction); dataStream.addSink(sinkFunction) - .name(TableConnectorUtil.generateRuntimeName(this.getClass(), getFieldNames())); + .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } @Override From 690cf7440d25354a53a56ce153d123d9a44459d6 Mon Sep 17 00:00:00 2001 From: xueyu <278006819@qq.com> Date: Mon, 21 Jan 2019 18:20:19 +0800 Subject: [PATCH 038/207] [FLINK-11046] [elasticsearch] Fix ElasticsearchSink deadlock when index failed with retry --- .../ElasticsearchApiCallBridge.java | 12 +++ .../ElasticsearchFailureHandlerIndexer.java | 88 +++++++++++++++++++ .../elasticsearch/ElasticsearchSinkBase.java | 19 +++- .../ElasticsearchSinkBaseTest.java | 6 +- 4 files changed, 119 insertions(+), 6 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index d3b774c8..e450485d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -93,6 +93,18 @@ default RequestIndexer createBulkProcessorIndexer( numPendingRequestsRef); } + /** + * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. + */ + default ElasticsearchFailureHandlerIndexer createFailureHandlerIndexer( + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new ElasticsearchFailureHandlerIndexer( + flushOnCheckpoint, + numPendingRequestsRef); + } + + /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java new file mode 100644 index 00000000..685b5d9c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java @@ -0,0 +1,88 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkRequest}. + * {@link ActionRequest ActionRequests} will be buffered before re-sending a bulk request to the Elasticsearch cluster. + */ + +@Internal +class ElasticsearchFailureHandlerIndexer implements RequestIndexer { + + private BulkRequest bulkRequest; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + ElasticsearchFailureHandlerIndexer(boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { + this.bulkRequest = new BulkRequest(); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkRequest.add(updateRequest); + } + } + + public BulkRequest getBulkRequest() { + return bulkRequest; + } + + public int numberOfActions() { + return bulkRequest.numberOfActions(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 4d0c0025..55d8854a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -166,6 +166,8 @@ public void setDelayMillis(long delayMillis) { /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient RequestIndexer requestIndexer; + private transient ElasticsearchFailureHandlerIndexer failureRequestIndexer; + // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink // ------------------------------------------------------------------------ @@ -296,12 +298,14 @@ public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = callBridge.createFailureHandlerIndexer(flushOnCheckpoint, numPendingRequests); } @Override public void invoke(T value) throws Exception { // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink checkErrorAndRethrow(); + reindexFailedRequest(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -380,6 +384,15 @@ private void checkErrorAndRethrow() { } } + private void reindexFailedRequest() { + if (failureRequestIndexer.numberOfActions() > 0) { + BulkRequest failedRequest = failureRequestIndexer.getBulkRequest(); + for (ActionRequest request: failedRequest.requests()) { + requestIndexer.add(request); + } + } + } + private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } @@ -400,9 +413,9 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon restStatus = itemResponse.getFailure().getStatus(); if (restStatus == null) { - failureHandler.onFailure(request.requests().get(i), failure, -1, requestIndexer); + failureHandler.onFailure(request.requests().get(i), failure, -1, failureRequestIndexer); } else { - failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), requestIndexer); + failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), failureRequestIndexer); } } } @@ -424,7 +437,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) try { for (ActionRequest action : request.requests()) { - failureHandler.onFailure(action, failure, -1, requestIndexer); + failureHandler.onFailure(action, failure, -1, failureRequestIndexer); } } catch (Throwable t) { // fail the sink and skip the rest of the items diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 322d64cb..fdfdece9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -366,9 +366,9 @@ public void go() throws Exception { // since the previous flush should have resulted in a request re-add from the failure handler, // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } +// while (snapshotThread.getState() != Thread.State.WAITING) { +// Thread.sleep(10); +// } // current number of pending request should be 1 due to the re-add Assert.assertEquals(1, sink.getNumPendingRequests()); From 2bc9fba9dce0efdbf4c6563934fb5e33d285d3d4 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 18:11:25 +0800 Subject: [PATCH 039/207] [FLINK-11046] [elasticsearch] Simplify failure handler indexer implementation Since we always use the original request indexer to re-index failed requests, and that indexer already keeps track of the number of pending actions, the ElasticsearchFailureHandlerIndexer does not need to keep track of the number of pending records by itself. --- .../BufferingNoOpRequestIndexer.java | 76 ++++++++++++++++ .../ElasticsearchApiCallBridge.java | 12 --- .../ElasticsearchFailureHandlerIndexer.java | 88 ------------------- .../elasticsearch/ElasticsearchSinkBase.java | 20 ++--- 4 files changed, 82 insertions(+), 114 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java new file mode 100644 index 00000000..e639b820 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java @@ -0,0 +1,76 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests} + * before re-sending them to the Elasticsearch cluster upon request. + */ +@Internal +@NotThreadSafe +class BufferingNoOpRequestIndexer implements RequestIndexer { + + private List bufferedRequests; + + BufferingNoOpRequestIndexer() { + this.bufferedRequests = new ArrayList<>(10); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + Collections.addAll(bufferedRequests, deleteRequests); + } + + @Override + public void add(IndexRequest... indexRequests) { + Collections.addAll(bufferedRequests, indexRequests); + } + + @Override + public void add(UpdateRequest... updateRequests) { + Collections.addAll(bufferedRequests, updateRequests); + } + + void processBufferedRequests(RequestIndexer actualIndexer) { + for (ActionRequest request : bufferedRequests) { + if (request instanceof IndexRequest) { + actualIndexer.add((IndexRequest) request); + } else if (request instanceof DeleteRequest) { + actualIndexer.add((DeleteRequest) request); + } else if (request instanceof UpdateRequest) { + actualIndexer.add((UpdateRequest) request); + } + } + + bufferedRequests.clear(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index e450485d..d3b774c8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -93,18 +93,6 @@ default RequestIndexer createBulkProcessorIndexer( numPendingRequestsRef); } - /** - * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. - */ - default ElasticsearchFailureHandlerIndexer createFailureHandlerIndexer( - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new ElasticsearchFailureHandlerIndexer( - flushOnCheckpoint, - numPendingRequestsRef); - } - - /** * Perform any necessary state cleanup. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java deleted file mode 100644 index 685b5d9c..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchFailureHandlerIndexer.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.annotation.Internal; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.bulk.BulkRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; - -import java.util.concurrent.atomic.AtomicLong; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Implementation of a {@link RequestIndexer}, using a {@link BulkRequest}. - * {@link ActionRequest ActionRequests} will be buffered before re-sending a bulk request to the Elasticsearch cluster. - */ - -@Internal -class ElasticsearchFailureHandlerIndexer implements RequestIndexer { - - private BulkRequest bulkRequest; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; - - ElasticsearchFailureHandlerIndexer(boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { - this.bulkRequest = new BulkRequest(); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } - - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(deleteRequest); - } - } - - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(indexRequest); - } - } - - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkRequest.add(updateRequest); - } - } - - public BulkRequest getBulkRequest() { - return bulkRequest; - } - - public int numberOfActions() { - return bulkRequest.numberOfActions(); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 55d8854a..e50140dd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -166,7 +166,8 @@ public void setDelayMillis(long delayMillis) { /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ private transient RequestIndexer requestIndexer; - private transient ElasticsearchFailureHandlerIndexer failureRequestIndexer; + /** Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed requests. */ + private transient BufferingNoOpRequestIndexer failureRequestIndexer; // ------------------------------------------------------------------------ // Internals for the Flink Elasticsearch Sink @@ -298,15 +299,15 @@ public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); - failureRequestIndexer = callBridge.createFailureHandlerIndexer(flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = new BufferingNoOpRequestIndexer(); } @Override public void invoke(T value) throws Exception { // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink checkErrorAndRethrow(); - reindexFailedRequest(); + failureRequestIndexer.processBufferedRequests(requestIndexer); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -320,10 +321,10 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { checkErrorAndRethrow(); if (flushOnCheckpoint) { - do { + while (numPendingRequests.get() != 0) { bulkProcessor.flush(); checkErrorAndRethrow(); - } while (numPendingRequests.get() != 0); + } } } @@ -384,15 +385,6 @@ private void checkErrorAndRethrow() { } } - private void reindexFailedRequest() { - if (failureRequestIndexer.numberOfActions() > 0) { - BulkRequest failedRequest = failureRequestIndexer.getBulkRequest(); - for (ActionRequest request: failedRequest.requests()) { - requestIndexer.add(request); - } - } - } - private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } From 5a7861dd42d3b007a2fa8cba454233d42d5609c9 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 19:12:46 +0800 Subject: [PATCH 040/207] [FLINK-11046] [elasticsearch] Always re-index buffered failed requests on snapshot This ensures that at-least-once semantics is not broken due to the changes to the failure handle request indexer. This closes #7576. --- .../elasticsearch/ElasticsearchSinkBase.java | 14 ++++++++------ .../elasticsearch/ElasticsearchSinkBaseTest.java | 6 +++--- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index e50140dd..5399c98e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -304,10 +304,7 @@ public void open(Configuration parameters) throws Exception { @Override public void invoke(T value) throws Exception { - // if bulk processor callbacks have previously reported an error, we rethrow the error and fail the sink - checkErrorAndRethrow(); - - failureRequestIndexer.processBufferedRequests(requestIndexer); + checkAsyncErrorsAndRequests(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } @@ -318,12 +315,12 @@ public void initializeState(FunctionInitializationContext context) throws Except @Override public void snapshotState(FunctionSnapshotContext context) throws Exception { - checkErrorAndRethrow(); + checkAsyncErrorsAndRequests(); if (flushOnCheckpoint) { while (numPendingRequests.get() != 0) { bulkProcessor.flush(); - checkErrorAndRethrow(); + checkAsyncErrorsAndRequests(); } } } @@ -385,6 +382,11 @@ private void checkErrorAndRethrow() { } } + private void checkAsyncErrorsAndRequests() { + checkErrorAndRethrow(); + failureRequestIndexer.processBufferedRequests(requestIndexer); + } + private class BulkProcessorListener implements BulkProcessor.Listener { @Override public void beforeBulk(long executionId, BulkRequest request) { } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index fdfdece9..322d64cb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -366,9 +366,9 @@ public void go() throws Exception { // since the previous flush should have resulted in a request re-add from the failure handler, // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush -// while (snapshotThread.getState() != Thread.State.WAITING) { -// Thread.sleep(10); -// } + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } // current number of pending request should be 1 due to the re-add Assert.assertEquals(1, sink.getNumPendingRequests()); From 63e5f66dff80057fc34828bf2cff6ce25970c5fc Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 9 Feb 2019 19:15:32 +0800 Subject: [PATCH 041/207] [hotfix] [elasticsearch] Replace implementation of deprecated invoke method in ElasticsearchSinkBase --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 5399c98e..da06d1c3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -203,7 +203,7 @@ public void setDelayMillis(long delayMillis) { private final AtomicReference failureThrowable = new AtomicReference<>(); public ElasticsearchSinkBase( - ElasticsearchApiCallBridge callBridge, + ElasticsearchApiCallBridge callBridge, Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction, ActionRequestFailureHandler failureHandler) { @@ -303,7 +303,7 @@ public void open(Configuration parameters) throws Exception { } @Override - public void invoke(T value) throws Exception { + public void invoke(T value, Context context) throws Exception { checkAsyncErrorsAndRequests(); elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); } From 8e2d4c78db63da8cfcc9b3c5dc2635fe6dd65a7d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 25 Feb 2019 10:21:06 +0100 Subject: [PATCH 042/207] Update version to 1.9-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index ebd722ae..73aae639 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.8-SNAPSHOT + 1.9-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index b2a98865..de47b58a 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.8-SNAPSHOT + 1.9-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 95432f44..66ba7f0b 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.8-SNAPSHOT + 1.9-SNAPSHOT .. From 28f2518fecc5493997008f484cf03039debc9633 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 1 Feb 2019 23:23:06 +0800 Subject: [PATCH 043/207] [FLINK-11516][table-common] Port and move catalog transitive classes to flink-table-common This closes #7642. --- .../ElasticsearchUpsertTableSinkFactoryBase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index 63e9b342..f52de799 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -113,7 +113,7 @@ public List supportedProperties() { final List properties = new ArrayList<>(); // streaming properties - properties.add(UPDATE_MODE()); + properties.add(UPDATE_MODE); // Elasticsearch properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); @@ -150,7 +150,7 @@ public StreamTableSink> createStreamTableSink(Map Date: Wed, 13 Mar 2019 14:52:58 +0100 Subject: [PATCH 044/207] [FLINK-11901][build] Update NOTICE files with year 2019 This closes #7975. --- .../src/main/resources/META-INF/NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 9880ee92..4b595386 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch6 -Copyright 2014-2018 The Apache Software Foundation +Copyright 2014-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From e940ff8900941323cedaa3ca11283910d0df5fc2 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 11 Apr 2019 15:53:12 +0200 Subject: [PATCH 045/207] [FLINK-12151][es1] Remove Elasticsearch 1 connector --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- .../connectors/elasticsearch/ElasticsearchSinkBaseTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 73aae639..94761d91 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 1.7.1 + 2.3.5 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 322d64cb..65ae5ecf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -26,7 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; @@ -510,7 +510,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { if (mockItemFailure == null) { // the mock response for the item is success - mockResponses[i] = new BulkItemResponse(i, "opType", mock(ActionResponse.class)); + mockResponses[i] = new BulkItemResponse(i, "opType", mock(ActionWriteResponse.class)); } else { // the mock response for the item is failure mockResponses[i] = new BulkItemResponse(i, "opType", new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); From ba57ac6454b2b609f7f694013b75030bf30ea5cc Mon Sep 17 00:00:00 2001 From: hequn8128 Date: Wed, 24 Apr 2019 00:39:04 +0800 Subject: [PATCH 046/207] [FLINK-11067][table] port Schema, Rowtime, SchematicDescriptor, RegistrableDescriptor etc. --- .../ElasticsearchUpsertTableSinkFactoryBase.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index f52de799..4b1d2bcb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -81,9 +81,9 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_NAME; -import static org.apache.flink.table.descriptors.SchemaValidator.SCHEMA_TYPE; +import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; @@ -136,8 +136,8 @@ public List supportedProperties() { properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); // schema - properties.add(SCHEMA() + ".#." + SCHEMA_TYPE()); - properties.add(SCHEMA() + ".#." + SCHEMA_NAME()); + properties.add(SCHEMA + ".#." + SCHEMA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_NAME); // format wildcard properties.add(FORMAT + ".*"); @@ -151,7 +151,7 @@ public StreamTableSink> createStreamTableSink(Map Date: Mon, 24 Jun 2019 16:08:25 +0800 Subject: [PATCH 047/207] [hotfix] [connectors] Fix shadowed NPE in elasticsearch sink connector This closes #8849. --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index da06d1c3..96f44314 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -427,7 +427,7 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure.getCause()); + LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); try { for (ActionRequest action : request.requests()) { From 84adcb78d49a9a869c85f7cee34dcb7d7b88072a Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 25 Jun 2019 15:24:40 +0200 Subject: [PATCH 048/207] [FLINK-12985][core][streaming] Rename StreamTransformation to org.apache.flink.api.dag.Transformation This name better represent the fact that Transformation also supports batch cases. Moreover it makes the structure of the flink-core cleaner. --- ...Elasticsearch6UpsertTableSinkFactoryTest.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 3ca9022b..ce9bde21 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -22,14 +22,13 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.transformations.StreamTransformation; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; @@ -194,7 +193,7 @@ private static class DataStreamMock extends DataStream> { public SinkFunction sinkFunction; public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation> outType) { - super(environment, new StreamTransformationMock("name", outType, 1)); + super(environment, new TransformationMock("name", outType, 1)); } @Override @@ -204,19 +203,14 @@ public DataStreamSink> addSink(SinkFunction> { + private static class TransformationMock extends Transformation> { - public StreamTransformationMock(String name, TypeInformation> outputType, int parallelism) { + public TransformationMock(String name, TypeInformation> outputType, int parallelism) { super(name, outputType, parallelism); } @Override - public void setChainingStrategy(ChainingStrategy strategy) { - // do nothing - } - - @Override - public Collection> getTransitivePredecessors() { + public Collection> getTransitivePredecessors() { return null; } } From 23069c6f8d3d08bf68df7fa29fe41474ea3f3288 Mon Sep 17 00:00:00 2001 From: ifndef-SleePy Date: Tue, 25 Jun 2019 11:54:27 +0800 Subject: [PATCH 049/207] [FLINK-12961][datastream] Add internal StreamExecutionEnvironment.execute(StreamGraph) --- .../Elasticsearch6UpsertTableSinkFactoryTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index ce9bde21..a9f08149 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -29,6 +29,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; @@ -183,7 +184,7 @@ protected ElasticsearchSink.Builder> createBuilder( private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { @Override - public JobExecutionResult execute(String jobName) { + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { throw new UnsupportedOperationException(); } } From 1eda4e956f07341aad5b0869d4d267f4c1cd9fef Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Fri, 12 Jul 2019 14:25:23 +0800 Subject: [PATCH 050/207] Update version to 1.10-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 94761d91..96700235 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.9-SNAPSHOT + 1.10-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index de47b58a..13bc8155 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.9-SNAPSHOT + 1.10-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 66ba7f0b..090fc981 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.9-SNAPSHOT + 1.10-SNAPSHOT .. From 1c593fcc1375b98f90035033fa04d57e67148d3d Mon Sep 17 00:00:00 2001 From: chenqi Date: Sat, 20 Jul 2019 23:22:43 +0800 Subject: [PATCH 051/207] [FLINK-13341][table][connectors] StreamTableSink#consumeDataStream returns DataStreamSink when using blink planner. This closes #9186 --- .../ElasticsearchUpsertTableSinkBase.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 045d3a6c..427bb6e8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; @@ -168,7 +169,7 @@ public TypeInformation getRecordType() { } @Override - public void emitDataStream(DataStream> dataStream) { + public DataStreamSink consumeDataStream(DataStream> dataStream) { final ElasticsearchUpsertSinkFunction upsertFunction = new ElasticsearchUpsertSinkFunction( index, @@ -184,10 +185,15 @@ public void emitDataStream(DataStream> dataStream) { failureHandler, sinkOptions, upsertFunction); - dataStream.addSink(sinkFunction) + return dataStream.addSink(sinkFunction) .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } + @Override + public void emitDataStream(DataStream> dataStream) { + consumeDataStream(dataStream); + } + @Override public TypeInformation> getOutputType() { return Types.TUPLE(Types.BOOLEAN, getRecordType()); From 13062a71cefe54cdc01bbb8d802b7aba1bb0a3e7 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 2 Aug 2019 14:21:34 +0800 Subject: [PATCH 052/207] [FLINK-13544][connectors] Set parallelism of table sink operator to input transformation parallelism This closes #9332 --- .../elasticsearch/ElasticsearchUpsertTableSinkBase.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 427bb6e8..c2c5181d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -186,6 +186,7 @@ public DataStreamSink consumeDataStream(DataStream> data sinkOptions, upsertFunction); return dataStream.addSink(sinkFunction) + .setParallelism(dataStream.getParallelism()) .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } From 4f292b3b3e5148943c06c7b1a5936cc973826c88 Mon Sep 17 00:00:00 2001 From: "zhuzhu.zz" Date: Thu, 12 Sep 2019 11:18:58 +0800 Subject: [PATCH 053/207] [hotfix][build] Use parent's maven-surefire-plugin version in elasticsearch6 connector --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 13bc8155..cc0bbf25 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -195,7 +195,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - 2.12.2 true From 1b10655963faa02b49d6c02baea4f10fcfe252aa Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 7 Oct 2019 10:40:58 +0200 Subject: [PATCH 054/207] [FLINK-14334][es][docs] Use ExceptionUtils#findThrowable This closes #9849. --- .../connectors/elasticsearch/ActionRequestFailureHandler.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index 260f80e5..1514aba6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -36,10 +36,10 @@ * * @Override * void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - * if (ExceptionUtils.containsThrowable(failure, EsRejectedExecutionException.class)) { + * if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { * // full queue; re-add document for indexing * indexer.add(action); - * } else if (ExceptionUtils.containsThrowable(failure, ElasticsearchParseException.class)) { + * } else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) { * // malformed document; simply drop request without failing sink * } else { * // for all other failures, fail the sink; From 4259a05f35ffa36786a381ed7b94bc50f184e88b Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 18 Sep 2019 20:05:05 +0800 Subject: [PATCH 055/207] [FLINK-13025] Elasticsearch 7.x support --- .../ElasticsearchUpsertTableSinkBase.java | 3 + .../descriptors/ElasticsearchValidator.java | 4 +- .../flink-connector-elasticsearch6/pom.xml | 8 - .../flink-connector-elasticsearch7/pom.xml | 190 ++++++++++++ .../Elasticsearch7ApiCallBridge.java | 143 +++++++++ .../Elasticsearch7BulkProcessorIndexer.java | 85 ++++++ .../elasticsearch7/ElasticsearchSink.java | 238 +++++++++++++++ .../elasticsearch7/RestClientFactory.java | 40 +++ ...eddedElasticsearchNodeEnvironmentImpl.java | 79 +++++ .../ElasticsearchSinkITCase.java | 100 ++++++ .../src/test/resources/log4j-test.properties | 24 ++ .../pom.xml | 205 +++++++++++++ .../Elasticsearch7UpsertTableSink.java | 285 ++++++++++++++++++ .../Elasticsearch7UpsertTableSinkFactory.java | 76 +++++ .../src/main/resources/META-INF/NOTICE | 45 +++ ....apache.flink.table.factories.TableFactory | 16 + ...sticsearch7UpsertTableSinkFactoryTest.java | 217 +++++++++++++ .../src/test/resources/log4j-test.properties | 24 ++ 18 files changed, 1772 insertions(+), 10 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch7/pom.xml create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/pom.xml create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java create mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index c2c5181d..eba7ead1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -377,6 +377,7 @@ public interface RequestFactory extends Serializable { /** * Creates an update request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ UpdateRequest createUpdateRequest( String index, @@ -387,6 +388,7 @@ UpdateRequest createUpdateRequest( /** * Creates an index request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ IndexRequest createIndexRequest( String index, @@ -396,6 +398,7 @@ IndexRequest createIndexRequest( /** * Creates a delete request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. */ DeleteRequest createDeleteRequest( String index, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java index 9d1a926e..ec48557c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; @@ -36,6 +35,7 @@ public class ElasticsearchValidator extends ConnectorDescriptorValidator { public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; public static final String CONNECTOR_VERSION_VALUE_6 = "6"; + public static final String CONNECTOR_VERSION_VALUE_7 = "7"; public static final String CONNECTOR_HOSTS = "connector.hosts"; public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; public static final String CONNECTOR_HOSTS_PORT = "port"; @@ -79,7 +79,7 @@ private void validateVersion(DescriptorProperties properties) { properties.validateEnumValues( CONNECTOR_VERSION, false, - Collections.singletonList(CONNECTOR_VERSION_VALUE_6)); + Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7)); } private void validateHosts(DescriptorProperties properties) { diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index cc0bbf25..5bdf889e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -93,14 +93,6 @@ under the License. provided true - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - provided - true - diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml new file mode 100644 index 00000000..3c68eac4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -0,0 +1,190 @@ + + + + + 4.0.0 + + + flink-connectors + org.apache.flink + 1.10-SNAPSHOT + .. + + + flink-connector-elasticsearch7_${scala.binary.version} + flink-connector-elasticsearch7 + + jar + + + + 7.3.2 + + + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + + org.elasticsearch.client + transport + ${elasticsearch.version} + test + + + + org.elasticsearch.plugin + transport-netty4-client + ${elasticsearch.version} + test + + + + + + org.apache.logging.log4j + log4j-core + 2.9.1 + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.2 + + + true + + + org.apache.logging.log4j:log4j-to-slf4j + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java new file mode 100644 index 00000000..c569f3b8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -0,0 +1,143 @@ +/* + * 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.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. + */ +@Internal +public class Elasticsearch7ApiCallBridge implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class); + + /** + * User-provided HTTP Host. + */ + private final List httpHosts; + + /** + * The factory to configure the rest client. + */ + private final RestClientFactory restClientFactory; + + Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) throws IOException { + RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!rhlClient.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); + } + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder((request, bulkListener) -> client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch7BulkProcessorIndexer( + bulkProcessor, + flushOnCheckpoint, + numPendingRequestsRef); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java new file mode 100644 index 00000000..0b7a4374 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.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 org.apache.flink.streaming.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. + * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * + *

Note: This class is binary compatible to Elasticsearch 7. + */ +@Internal +class Elasticsearch7BulkProcessorIndexer implements RequestIndexer { + + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; + + Elasticsearch7BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } + + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java new file mode 100644 index 00000000..5b874e29 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -0,0 +1,238 @@ +/* + * 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.connectors.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Elasticsearch 7.x sink that requests multiple {@link ActionRequest ActionRequests} + * against a cluster for each incoming element. + * + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. + * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. + * This will buffer elements before sending a request to the cluster. The behaviour of the + * {@code BulkProcessor} can be configured using these config keys: + *

    + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds + *
+ * + *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of + * {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + */ +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super(new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + */ + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + */ + public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maxinum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, + "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) && + Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) && + Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) && + Objects.equals(failureHandler, builder.failureHandler) && + Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java new file mode 100644 index 00000000..6001f43c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java @@ -0,0 +1,40 @@ +/* + * 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.connectors.elasticsearch7; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally + * used in the {@link ElasticsearchSink}. + */ +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); + +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java new file mode 100644 index 00000000..6f051a38 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -0,0 +1,79 @@ +/* + * 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.connectors.elasticsearch; + +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase; + +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.InternalSettingsPreparer; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.transport.Netty4Plugin; + +import java.io.File; +import java.util.Collections; + +/** + * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 7. + * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. + */ +public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { + + private Node node; + + @Override + public void start(File tmpDataFolder, String clusterName) throws Exception { + if (node == null) { + Settings settings = Settings.builder() + .put("cluster.name", clusterName) + .put("http.cors.enabled", true) + .put("path.home", tmpDataFolder.getParent()) + .put("path.data", tmpDataFolder.getAbsolutePath()) + .build(); + + node = new PluginNode(settings); + node.start(); + } + } + + @Override + public void close() throws Exception { + if (node != null && !node.isClosed()) { + node.close(); + node = null; + } + } + + @Override + public Client getClient() { + if (node != null && !node.isClosed()) { + return node.client(); + } else { + return null; + } + } + + private static class PluginNode extends Node { + public PluginNode(Settings settings) { + super(InternalSettingsPreparer.prepareEnvironment(settings, Collections.emptyMap(), null, () -> "node1"), Collections.>singletonList(Netty4Plugin.class), true); + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java new file mode 100644 index 00000000..c8de4473 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -0,0 +1,100 @@ +/* + * 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.connectors.elasticsearch7; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +/** + * IT cases for the {@link ElasticsearchSink}. + * + *

The Elasticsearch ITCases for 7.x CANNOT be executed in the IDE directly, since it is required that the + * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded + * node used in the tests to work properly. + */ +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testNullAddresses() throws Exception { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost(ipAddress, 9200, "http")); + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties new file mode 100644 index 00000000..fcd86546 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties @@ -0,0 +1,24 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml new file mode 100644 index 00000000..d1d81865 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -0,0 +1,205 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.10-SNAPSHOT + .. + + + flink-sql-connector-elasticsearch7_${scala.binary.version} + flink-sql-connector-elasticsearch7 + + jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + org.apache.flink + flink-connector-elasticsearch7_${scala.binary.version} + ${project.version} + + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + false + + + *:* + + + + com.carrotsearch:hppc + com.tdunning:t-digest + joda-time:joda-time + net.sf.jopt-simple:jopt-simple + org.elasticsearch:jna + org.hdrhistogram:HdrHistogram + org.yaml:snakeyaml + + + + + + org.elasticsearch:elasticsearch + + config/** + modules.txt + plugins.txt + org/joda/** + + + + org.elasticsearch.client:elasticsearch-rest-high-level-client + + forbidden/** + + + + org.apache.httpcomponents:httpclient + + mozilla/** + + + + org.apache.lucene:lucene-analyzers-common + + org/tartarus/** + + + + *:* + + + META-INF/versions/** + META-INF/services/com.fasterxml.** + META-INF/services/org.apache.lucene.** + META-INF/services/org.elasticsearch.** + META-INF/LICENSE.txt + + + + + + + org.apache.commons + org.apache.flink.elasticsearch7.shaded.org.apache.commons + + + org.apache.http + org.apache.flink.elasticsearch7.shaded.org.apache.http + + + org.apache.lucene + org.apache.flink.elasticsearch7.shaded.org.apache.lucene + + + org.elasticsearch + org.apache.flink.elasticsearch7.shaded.org.elasticsearch + + + org.apache.logging + org.apache.flink.elasticsearch7.shaded.org.apache.logging + + + com.fasterxml.jackson + org.apache.flink.elasticsearch7.shaded.com.fasterxml.jackson + + + + + + + + + + diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java new file mode 100644 index 00000000..b6cf75c9 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java @@ -0,0 +1,285 @@ +/* + * 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.sql.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_DELAY; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_ENABLED; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_RETRIES; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_TYPE; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_INTERVAL; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_ACTIONS; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_SIZE; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.DISABLE_FLUSH_ON_CHECKPOINT; +import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; + +/** + * Version-specific upsert table sink for Elasticsearch 7. + */ +@Internal +public class Elasticsearch7UpsertTableSink extends ElasticsearchUpsertTableSinkBase { + + @VisibleForTesting + static final RequestFactory UPDATE_REQUEST_FACTORY = + new Elasticsearch7RequestFactory(); + + public Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + "", + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @VisibleForTesting + Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @Override + protected ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertSinkFunction) { + + final List httpHosts = hosts.stream() + .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) + .collect(Collectors.toList()); + + final ElasticsearchSink.Builder> builder = createBuilder(upsertSinkFunction, httpHosts); + + builder.setFailureHandler(failureHandler); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) + .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) + .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) + .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) + .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) + .ifPresent(v -> builder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) + .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) + .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); + + builder.setRestClientFactory( + new DefaultRestClientFactory(sinkOptions.get(REST_PATH_PREFIX))); + + final ElasticsearchSink> sink = builder.build(); + + Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) + .ifPresent(v -> { + if (Boolean.valueOf(v)) { + sink.disableFlushOnCheckpoint(); + } + }); + + return sink; + } + + @VisibleForTesting + ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, + List httpHosts) { + return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** + * Serializable {@link RestClientFactory} used by the sink. + */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + XContentType contentType, + byte[] document) { + return new IndexRequest(index) + .source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java new file mode 100644 index 00000000..ca2f5449 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java @@ -0,0 +1,76 @@ +/* + * 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.sql.connectors.elasticsearch7; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryBase; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.types.Row; + +import org.elasticsearch.common.xcontent.XContentType; + +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; + +/** + * Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 7. + */ +@Internal +public class Elasticsearch7UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { + + @Override + protected String elasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } + + @Override + protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE new file mode 100644 index 00000000..2c7e1c03 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -0,0 +1,45 @@ +flink-sql-connector-elasticsearch7 +Copyright 2014-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) + +- com.fasterxml.jackson.core:jackson-core:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- commons-codec:commons-codec:1.10 +- commons-logging:commons-logging:1.1.3 +- org.apache.httpcomponents:httpasyncclient:4.1.2 +- org.apache.httpcomponents:httpclient:4.5.3 +- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpcore-nio:4.4.5 +- org.apache.logging.log4j:log4j-api:2.9.1 +- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 +- org.apache.lucene:lucene-analyzers-common:7.3.1 +- org.apache.lucene:lucene-backward-codecs:7.3.1 +- org.apache.lucene:lucene-core:7.3.1 +- org.apache.lucene:lucene-grouping:7.3.1 +- org.apache.lucene:lucene-highlighter:7.3.1 +- org.apache.lucene:lucene-join:7.3.1 +- org.apache.lucene:lucene-memory:7.3.1 +- org.apache.lucene:lucene-misc:7.3.1 +- org.apache.lucene:lucene-queries:7.3.1 +- org.apache.lucene:lucene-queryparser:7.3.1 +- org.apache.lucene:lucene-sandbox:7.3.1 +- org.apache.lucene:lucene-spatial:7.3.1 +- org.apache.lucene:lucene-spatial-extras:7.3.1 +- org.apache.lucene:lucene-spatial3d:7.3.1 +- org.apache.lucene:lucene-suggest:7.3.1 +- org.elasticsearch:elasticsearch:7.3.2 +- org.elasticsearch:elasticsearch-cli:7.3.2 +- org.elasticsearch:elasticsearch-core:7.3.2 +- org.elasticsearch:elasticsearch-secure-sm:7.3.2 +- org.elasticsearch:elasticsearch-x-content:7.3.2 +- org.elasticsearch.client:elasticsearch-rest-client:7.3.2 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.3.2 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.3.2 +- org.elasticsearch.plugin:parent-join-client:7.3.2 +- org.elasticsearch.plugin:rank-eval-client:7.3.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 00000000..fe10b08d --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.sql.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java new file mode 100644 index 00000000..a047b02f --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -0,0 +1,217 @@ +/* + * 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.sql.connectors.elasticsearch7; + +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.formats.json.JsonRowSerializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.ElasticsearchUpsertSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; + +import org.apache.http.HttpHost; +import org.elasticsearch.common.xcontent.XContentType; +import org.junit.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; +import static org.junit.Assert.assertEquals; + +/** + * Test for {@link Elasticsearch7UpsertTableSink} created by {@link Elasticsearch7UpsertTableSinkFactory}. + */ +public class Elasticsearch7UpsertTableSinkFactoryTest extends ElasticsearchUpsertTableSinkFactoryTestBase { + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + final TestElasticsearch7UpsertTableSink testSink = new TestElasticsearch7UpsertTableSink( + false, + schema, + Collections.singletonList(new Host(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), + ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, + ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + final DataStreamMock dataStreamMock = new DataStreamMock( + new StreamExecutionEnvironmentMock(), + Types.TUPLE(Types.BOOLEAN, schema.toRowType())); + + testSink.emitDataStream(dataStreamMock); + + final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( + Collections.singletonList(new HttpHost(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), + new ElasticsearchUpsertSinkFunction( + ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, + ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, + ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, + new int[0])); + expectedBuilder.setFailureHandler(new DummyFailureHandler()); + expectedBuilder.setBulkFlushBackoff(true); + expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + expectedBuilder.setBulkFlushBackoffDelay(123); + expectedBuilder.setBulkFlushBackoffRetries(3); + expectedBuilder.setBulkFlushInterval(100); + expectedBuilder.setBulkFlushMaxActions(1000); + expectedBuilder.setBulkFlushMaxSizeMb(1); + expectedBuilder.setRestClientFactory(new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); + + assertEquals(expectedBuilder, testSink.builder); + } + + @Override + protected String getElasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } + + @Override + protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + private static class TestElasticsearch7UpsertTableSink extends Elasticsearch7UpsertTableSink { + + public ElasticsearchSink.Builder> builder; + + public TestElasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, + List httpHosts) { + builder = super.createBuilder(upsertSinkFunction, httpHosts); + return builder; + } + } + + private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { + + @Override + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { + throw new UnsupportedOperationException(); + } + } + + private static class DataStreamMock extends DataStream> { + + public SinkFunction sinkFunction; + + public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation> outType) { + super(environment, new TransformationMock("name", outType, 1)); + } + + @Override + public DataStreamSink> addSink(SinkFunction> sinkFunction) { + this.sinkFunction = sinkFunction; + return super.addSink(sinkFunction); + } + } + + private static class TransformationMock extends Transformation> { + + public TransformationMock(String name, TypeInformation> outputType, int parallelism) { + super(name, outputType, parallelism); + } + + @Override + public Collection> getTransitivePredecessors() { + return null; + } + } +} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties new file mode 100644 index 00000000..fcd86546 --- /dev/null +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties @@ -0,0 +1,24 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n From 198f4dcbe919c6a899decfb69af4d2a3df7082f2 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Mon, 14 Oct 2019 13:05:45 +0200 Subject: [PATCH 056/207] [FLINK-13025] In ES7 SQL connector, update shading rules and NOTICE --- .../pom.xml | 3 ++ .../src/main/resources/META-INF/NOTICE | 45 +++++++++---------- 2 files changed, 25 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index d1d81865..63aa212d 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -122,6 +122,9 @@ under the License. joda-time:joda-time net.sf.jopt-simple:jopt-simple org.elasticsearch:jna + org.elasticsearch:elasticsearch-geo + org.elasticsearch.plugin:lang-mustache-client + com.github.spullara.mustache.java:compiler org.hdrhistogram:HdrHistogram org.yaml:snakeyaml diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 2c7e1c03..80866c7c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -6,33 +6,32 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- com.fasterxml.jackson.core:jackson-core:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 -- org.apache.httpcomponents:httpasyncclient:4.1.2 +- org.apache.httpcomponents:httpasyncclient:4.1.4 - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.httpcomponents:httpcore-nio:4.4.5 -- org.apache.logging.log4j:log4j-api:2.9.1 -- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 -- org.apache.lucene:lucene-analyzers-common:7.3.1 -- org.apache.lucene:lucene-backward-codecs:7.3.1 -- org.apache.lucene:lucene-core:7.3.1 -- org.apache.lucene:lucene-grouping:7.3.1 -- org.apache.lucene:lucene-highlighter:7.3.1 -- org.apache.lucene:lucene-join:7.3.1 -- org.apache.lucene:lucene-memory:7.3.1 -- org.apache.lucene:lucene-misc:7.3.1 -- org.apache.lucene:lucene-queries:7.3.1 -- org.apache.lucene:lucene-queryparser:7.3.1 -- org.apache.lucene:lucene-sandbox:7.3.1 -- org.apache.lucene:lucene-spatial:7.3.1 -- org.apache.lucene:lucene-spatial-extras:7.3.1 -- org.apache.lucene:lucene-spatial3d:7.3.1 -- org.apache.lucene:lucene-suggest:7.3.1 +- org.apache.httpcomponents:httpcore-nio:4.4.11 +- org.apache.logging.log4j:log4j-api:2.11.1 +- org.apache.lucene:lucene-analyzers-common:8.1.0 +- org.apache.lucene:lucene-backward-codecs:8.1.0 +- org.apache.lucene:lucene-core:8.1.0 +- org.apache.lucene:lucene-grouping:8.1.0 +- org.apache.lucene:lucene-highlighter:8.1.0 +- org.apache.lucene:lucene-join:8.1.0 +- org.apache.lucene:lucene-memory:8.1.0 +- org.apache.lucene:lucene-misc:8.1.0 +- org.apache.lucene:lucene-queries:8.1.0 +- org.apache.lucene:lucene-queryparser:8.1.0 +- org.apache.lucene:lucene-sandbox:8.1.0 +- org.apache.lucene:lucene-spatial:8.1.0 +- org.apache.lucene:lucene-spatial-extras:8.1.0 +- org.apache.lucene:lucene-spatial3d:8.1.0 +- org.apache.lucene:lucene-suggest:8.1.0 - org.elasticsearch:elasticsearch:7.3.2 - org.elasticsearch:elasticsearch-cli:7.3.2 - org.elasticsearch:elasticsearch-core:7.3.2 From 617bdf6bda2fc5106cafb8b8b339dfd52e9c036b Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 15 Oct 2019 21:01:59 +0800 Subject: [PATCH 057/207] [hotfix][kafka][es] Add missing @PublicEvolving annotation on Kafka and Elasticsearch descriptor class --- .../java/org/apache/flink/table/descriptors/Elasticsearch.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java index fa07821e..15614c94 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -18,6 +18,7 @@ package org.apache.flink.table.descriptors; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.MemorySize; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; @@ -54,6 +55,7 @@ /** * Connector descriptor for the Elasticsearch search engine. */ +@PublicEvolving public class Elasticsearch extends ConnectorDescriptor { private DescriptorProperties internalProperties = new DescriptorProperties(true); From abaa6bf2631d6dc06bd68d9fbe1007cc9161dd33 Mon Sep 17 00:00:00 2001 From: yanghua Date: Tue, 22 Oct 2019 20:58:02 +0800 Subject: [PATCH 058/207] [FLINK-14395] Refactor ES 7 connectors to make them keep consistency with es 6 connectors --- .../flink-connector-elasticsearch7/pom.xml | 14 ++++- .../Elasticsearch7UpsertTableSink.java | 4 +- .../Elasticsearch7UpsertTableSinkFactory.java | 2 +- ....apache.flink.table.factories.TableFactory | 2 +- ...sticsearch7UpsertTableSinkFactoryTest.java | 3 +- .../pom.xml | 54 ------------------- .../src/main/resources/META-INF/NOTICE | 1 + .../src/test/resources/log4j-test.properties | 24 --------- 8 files changed, 18 insertions(+), 86 deletions(-) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java (97%) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java (97%) rename flink-connectors/{flink-sql-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory (89%) rename flink-connectors/{flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql => flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming}/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java (98%) delete mode 100644 flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 3c68eac4..0b5fda5e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -71,6 +71,19 @@ under the License. ${elasticsearch.version} + + + + org.apache.logging.log4j + log4j-to-slf4j + 2.11.1 + + @@ -174,7 +187,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - 2.12.2 true diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java similarity index 97% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java rename to flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java index b6cf75c9..7795fb8e 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; @@ -27,8 +27,6 @@ import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; -import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.types.Row; diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java similarity index 97% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java rename to flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java index ca2f5449..29208c80 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 89% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory index fe10b08d..5bcb91f3 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.sql.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory +org.apache.flink.streaming.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java similarity index 98% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java rename to flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index a047b02f..1b984523 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/sql/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.sql.connectors.elasticsearch7; +package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.serialization.SerializationSchema; @@ -37,7 +37,6 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; import org.apache.flink.table.api.TableSchema; import org.apache.flink.types.Row; diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 63aa212d..096f4126 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -36,65 +36,11 @@ under the License. jar - - org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} - ${project.version} - - - - org.elasticsearch - elasticsearch - - - - org.apache.flink flink-connector-elasticsearch7_${scala.binary.version} ${project.version} - - - - - org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} - ${project.version} - provided - true - - - - org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} - ${project.version} - - - org.elasticsearch - elasticsearch - - - test-jar - test - - - - - org.apache.flink - flink-json - ${project.version} - test - - - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 80866c7c..7cc11198 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -17,6 +17,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpcore:4.4.6 - org.apache.httpcomponents:httpcore-nio:4.4.11 - org.apache.logging.log4j:log4j-api:2.11.1 +- org.apache.logging.log4j:log4j-to-slf4j:2.11.1 - org.apache.lucene:lucene-analyzers-common:8.1.0 - org.apache.lucene:lucene-backward-codecs:8.1.0 - org.apache.lucene:lucene-core:8.1.0 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties deleted file mode 100644 index fcd86546..00000000 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/test/resources/log4j-test.properties +++ /dev/null @@ -1,24 +0,0 @@ -################################################################################ -# 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. -################################################################################ - -log4j.rootLogger=INFO, testlogger - -log4j.appender.testlogger=org.apache.log4j.ConsoleAppender -log4j.appender.testlogger.target=System.err -log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n From 24f021c2792923f344db4d8c0ca3a64e9341cf27 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 20 Nov 2019 10:35:03 +0100 Subject: [PATCH 059/207] [FLINK-14104][build] Pin jackson version to 2.10.1 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 4b595386..20dec329 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.8.10 +- com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 7cc11198..7a0ba933 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.8.11 +- com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.11 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.11 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 From f4ed19ef9fc1d8f803a591fdc2d43bdf19325105 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 3 Dec 2019 20:55:04 +0800 Subject: [PATCH 060/207] [FLINK-14645][connectors] Support new schema properties for SQL connectors --- ...lasticsearchUpsertTableSinkFactoryBase.java | 2 ++ ...icsearchUpsertTableSinkFactoryTestBase.java | 18 +++++++++--------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index 4b1d2bcb..ebf641e5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -82,6 +82,7 @@ import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; @@ -136,6 +137,7 @@ public List supportedProperties() { properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); // schema + properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_NAME); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index 0cc282c0..fce89833 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -22,8 +22,8 @@ import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.Types; import org.apache.flink.table.descriptors.Elasticsearch; import org.apache.flink.table.descriptors.Json; import org.apache.flink.table.descriptors.Schema; @@ -106,10 +106,10 @@ public void testTableSink() { .deriveSchema()) .withSchema( new Schema() - .field(FIELD_KEY, Types.LONG()) - .field(FIELD_FRUIT_NAME, Types.STRING()) - .field(FIELD_COUNT, Types.DECIMAL()) - .field(FIELD_TS, Types.SQL_TIMESTAMP())) + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3))) .inUpsertMode(); final Map propertiesMap = testDesc.toProperties(); @@ -121,10 +121,10 @@ public void testTableSink() { protected TableSchema createTestSchema() { return TableSchema.builder() - .field(FIELD_KEY, Types.LONG()) - .field(FIELD_FRUIT_NAME, Types.STRING()) - .field(FIELD_COUNT, Types.DECIMAL()) - .field(FIELD_TS, Types.SQL_TIMESTAMP()) + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) .build(); } From 1fcd8352269a907a0781731004d26a1376f32dcf Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Mon, 9 Dec 2019 20:18:28 +0800 Subject: [PATCH 061/207] [FLINK-14649][kafka][es] Flatten all the connector properties keys to make it easy to configure in DDL (#10468) --- .../ElasticsearchUpsertTableSinkBase.java | 7 ++ ...asticsearchUpsertTableSinkFactoryBase.java | 24 +++-- .../table/descriptors/Elasticsearch.java | 19 ++-- .../descriptors/ElasticsearchValidator.java | 65 +++++++++++- ...csearchUpsertTableSinkFactoryTestBase.java | 98 +++++++++++++------ .../table/descriptors/ElasticsearchTest.java | 41 +++++--- 6 files changed, 184 insertions(+), 70 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index eba7ead1..e1136421 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -368,6 +368,13 @@ public int hashCode() { port, protocol); } + + @Override + public String toString() { + return protocol + "://" + + hostname + ":" + + port; + } } /** diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index ebf641e5..c9c7f23b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -79,6 +79,7 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.validateAndParseHostsString; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; import static org.apache.flink.table.descriptors.Schema.SCHEMA; @@ -117,6 +118,7 @@ public List supportedProperties() { properties.add(UPDATE_MODE); // Elasticsearch + properties.add(CONNECTOR_HOSTS); properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PORT); properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PROTOCOL); @@ -200,15 +202,19 @@ private DescriptorProperties getValidatedProperties(Map properti } private List getHosts(DescriptorProperties descriptorProperties) { - final List> hosts = descriptorProperties.getFixedIndexedProperties( - CONNECTOR_HOSTS, - Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL)); - return hosts.stream() - .map(host -> new Host( - descriptorProperties.getString(host.get(CONNECTOR_HOSTS_HOSTNAME)), - descriptorProperties.getInt(host.get(CONNECTOR_HOSTS_PORT)), - descriptorProperties.getString(host.get(CONNECTOR_HOSTS_PROTOCOL)))) - .collect(Collectors.toList()); + if (descriptorProperties.containsKey(CONNECTOR_HOSTS)) { + return validateAndParseHostsString(descriptorProperties); + } else { + final List> hosts = descriptorProperties.getFixedIndexedProperties( + CONNECTOR_HOSTS, + Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL)); + return hosts.stream() + .map(host -> new Host( + descriptorProperties.getString(host.get(CONNECTOR_HOSTS_HOSTNAME)), + descriptorProperties.getInt(host.get(CONNECTOR_HOSTS_PORT)), + descriptorProperties.getString(host.get(CONNECTOR_HOSTS_PROTOCOL)))) + .collect(Collectors.toList()); + } } private SerializationSchema getSerializationSchema(Map properties) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java index 15614c94..ccbd4c10 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -25,7 +25,6 @@ import org.apache.flink.util.Preconditions; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -44,9 +43,6 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_CLASS; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FLUSH_ON_CHECKPOINT; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_HOSTNAME; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PORT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PROTOCOL; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_INDEX; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; @@ -308,14 +304,13 @@ protected Map toConnectorProperties() { final DescriptorProperties properties = new DescriptorProperties(); properties.putProperties(internalProperties); - final List> hostValues = hosts.stream() - .map(host -> Arrays.asList(host.hostname, String.valueOf(host.port), host.protocol)) - .collect(Collectors.toList()); - properties.putIndexedFixedProperties( - CONNECTOR_HOSTS, - Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL), - hostValues); - + if (hosts.size() > 0) { + properties.putString( + CONNECTOR_HOSTS, + hosts.stream() + .map(Host::toString) + .collect(Collectors.joining(";"))); + } return properties.asMap(); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java index ec48557c..b8f23318 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -19,9 +19,16 @@ package org.apache.flink.table.descriptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.StringUtils; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.function.Consumer; @@ -83,11 +90,16 @@ private void validateVersion(DescriptorProperties properties) { } private void validateHosts(DescriptorProperties properties) { - final Map> hostsValidators = new HashMap<>(); - hostsValidators.put(CONNECTOR_HOSTS_HOSTNAME, (key) -> properties.validateString(key, false, 1)); - hostsValidators.put(CONNECTOR_HOSTS_PORT, (key) -> properties.validateInt(key, false, 0, 65535)); - hostsValidators.put(CONNECTOR_HOSTS_PROTOCOL, (key) -> properties.validateString(key, false, 1)); - properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); + if (properties.containsKey(CONNECTOR_HOSTS)) { + validateAndParseHostsString(properties); + } else { + final Map> hostsValidators = new HashMap<>(); + hostsValidators.put(CONNECTOR_HOSTS_HOSTNAME, (key) -> properties.validateString(key, false, 1)); + hostsValidators.put(CONNECTOR_HOSTS_PORT, (key) -> properties.validateInt(key, false, 0, 65535)); + hostsValidators.put(CONNECTOR_HOSTS_PROTOCOL, (key) -> properties.validateString(key, false, 1)); + + properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); + } } private void validateGeneralProperties(DescriptorProperties properties) { @@ -126,4 +138,47 @@ private void validateConnectionProperties(DescriptorProperties properties) { properties.validateInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, true, 1); properties.validateString(CONNECTOR_CONNECTION_PATH_PREFIX, true); } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+	 *     connector.hosts = http://host_name:9092;http://host_name:9093
+	 * 
+ */ + public static List validateAndParseHostsString(DescriptorProperties descriptorProperties) { + final List hostList = new ArrayList<>(); + + descriptorProperties.validateString(CONNECTOR_HOSTS, false, 1); + final String hostsStr = descriptorProperties.getString(CONNECTOR_HOSTS); + + final String[] hosts = hostsStr.split(";"); + final String validationExceptionMessage = "Properties '" + CONNECTOR_HOSTS + "' format should " + + "follow the format 'http://host_name:port', but is '" + hostsStr + "'."; + + if (hosts.length == 0) { + throw new ValidationException(validationExceptionMessage); + } + for (String host : hosts) { + try { + final URL url = new URL(host); + final String protocol = url.getProtocol(); + final String hostName = url.getHost(); + final int hostPort = url.getPort(); + + if (StringUtils.isNullOrWhitespaceOnly(protocol) || + StringUtils.isNullOrWhitespaceOnly(hostName) || + -1 == hostPort) { + throw new ValidationException(validationExceptionMessage); + } + + hostList.add(new Host(hostName, hostPort, protocol)); + } catch (MalformedURLException e) { + throw new ValidationException(validationExceptionMessage, e); + } + } + return hostList; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index fce89833..ea4644b0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -83,38 +83,45 @@ public void testTableSink() { createTestSinkOptions()); // construct table sink using descriptors and table sink factory + final Map elasticSearchProperties = createElasticSearchProperties(); + final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, elasticSearchProperties) + .createStreamTableSink(elasticSearchProperties); - final TestTableDescriptor testDesc = new TestTableDescriptor( - new Elasticsearch() - .version(getElasticsearchVersion()) - .host(HOSTNAME, PORT, SCHEMA) - .index(INDEX) - .documentType(DOC_TYPE) - .keyDelimiter(KEY_DELIMITER) - .keyNullLiteral(KEY_NULL_LITERAL) - .bulkFlushBackoffExponential() - .bulkFlushBackoffDelay(123L) - .bulkFlushBackoffMaxRetries(3) - .bulkFlushInterval(100L) - .bulkFlushMaxActions(1000) - .bulkFlushMaxSize("1 MB") - .failureHandlerCustom(DummyFailureHandler.class) - .connectionMaxRetryTimeout(100) - .connectionPathPrefix("/myapp")) - .withFormat( - new Json() - .deriveSchema()) - .withSchema( - new Schema() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3))) - .inUpsertMode(); + assertEquals(expectedSink, actualSink); + } - final Map propertiesMap = testDesc.toProperties(); - final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, propertiesMap) - .createStreamTableSink(propertiesMap); + @Test + public void testTableSinkWithLegacyProperties() { + // prepare parameters for Elasticsearch table sink + final TableSchema schema = createTestSchema(); + + final ElasticsearchUpsertTableSinkBase expectedSink = getExpectedTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + new JsonRowSerializationSchema(schema.toRowType()), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + // construct table sink using descriptors and table sink factory + final Map elasticSearchProperties = createElasticSearchProperties(); + + final Map legacyPropertiesMap = new HashMap<>(); + legacyPropertiesMap.putAll(elasticSearchProperties); + // use legacy properties + legacyPropertiesMap.remove("connector.hosts"); + + legacyPropertiesMap.put("connector.hosts.0.hostname", "host1"); + legacyPropertiesMap.put("connector.hosts.0.port", "1234"); + legacyPropertiesMap.put("connector.hosts.0.protocol", "https"); + + final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, legacyPropertiesMap) + .createStreamTableSink(legacyPropertiesMap); assertEquals(expectedSink, actualSink); } @@ -142,6 +149,37 @@ protected Map createTestSinkOptions() { return sinkOptions; } + protected Map createElasticSearchProperties() { + return new TestTableDescriptor( + new Elasticsearch() + .version(getElasticsearchVersion()) + .host(HOSTNAME, PORT, SCHEMA) + .index(INDEX) + .documentType(DOC_TYPE) + .keyDelimiter(KEY_DELIMITER) + .keyNullLiteral(KEY_NULL_LITERAL) + .bulkFlushBackoffExponential() + .bulkFlushBackoffDelay(123L) + .bulkFlushBackoffMaxRetries(3) + .bulkFlushInterval(100L) + .bulkFlushMaxActions(1000) + .bulkFlushMaxSize("1 MB") + .failureHandlerCustom(DummyFailureHandler.class) + .connectionMaxRetryTimeout(100) + .connectionPathPrefix("/myapp")) + .withFormat( + new Json() + .deriveSchema()) + .withSchema( + new Schema() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3))) + .inUpsertMode() + .toProperties(); + } + // -------------------------------------------------------------------------------------------- // For version-specific tests // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java index c97aa0e6..85326a63 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java @@ -28,6 +28,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; + /** * Tests for the {@link Elasticsearch} descriptor. */ @@ -48,10 +50,30 @@ public void testInvalidMemorySize() { addPropertyAndVerify(descriptors().get(1), "connector.bulk-flush.max-size", "12 bytes"); } + @Test(expected = ValidationException.class) + public void testInvalidProtocolInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "localhost:90"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + + @Test(expected = ValidationException.class) + public void testInvalidHostNameInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "http://:90"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + + @Test(expected = ValidationException.class) + public void testInvalidHostPortInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "http://localhost"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + @Override public List descriptors() { - final Descriptor minimumDesc = - new Elasticsearch() + final Descriptor minimumDesc = new Elasticsearch() .version("6") .host("localhost", 1234, "http") .index("MyIndex") @@ -94,9 +116,7 @@ public List> properties() { minimumDesc.put("connector.property-version", "1"); minimumDesc.put("connector.type", "elasticsearch"); minimumDesc.put("connector.version", "6"); - minimumDesc.put("connector.hosts.0.hostname", "localhost"); - minimumDesc.put("connector.hosts.0.port", "1234"); - minimumDesc.put("connector.hosts.0.protocol", "http"); + minimumDesc.put("connector.hosts", "http://localhost:1234"); minimumDesc.put("connector.index", "MyIndex"); minimumDesc.put("connector.document-type", "MyType"); @@ -104,12 +124,7 @@ public List> properties() { maximumDesc.put("connector.property-version", "1"); maximumDesc.put("connector.type", "elasticsearch"); maximumDesc.put("connector.version", "6"); - maximumDesc.put("connector.hosts.0.hostname", "host1"); - maximumDesc.put("connector.hosts.0.port", "1234"); - maximumDesc.put("connector.hosts.0.protocol", "https"); - maximumDesc.put("connector.hosts.1.hostname", "host2"); - maximumDesc.put("connector.hosts.1.port", "1234"); - maximumDesc.put("connector.hosts.1.protocol", "https"); + maximumDesc.put("connector.hosts", "https://host1:1234;https://host2:1234"); maximumDesc.put("connector.index", "MyIndex"); maximumDesc.put("connector.document-type", "MyType"); maximumDesc.put("connector.key-delimiter", "#"); @@ -128,9 +143,7 @@ public List> properties() { customDesc.put("connector.property-version", "1"); customDesc.put("connector.type", "elasticsearch"); customDesc.put("connector.version", "6"); - customDesc.put("connector.hosts.0.hostname", "localhost"); - customDesc.put("connector.hosts.0.port", "1234"); - customDesc.put("connector.hosts.0.protocol", "http"); + customDesc.put("connector.hosts", "http://localhost:1234"); customDesc.put("connector.index", "MyIndex"); customDesc.put("connector.document-type", "MyType"); customDesc.put("connector.flush-on-checkpoint", "false"); From f21e80e512353eeda08b8c55482e95808c5655f0 Mon Sep 17 00:00:00 2001 From: Gary Yao Date: Mon, 9 Dec 2019 17:28:45 +0100 Subject: [PATCH 062/207] Update version to 1.11-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 96700235..56d4b6b5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 5bdf889e..fa4799fa 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 0b5fda5e..c02037e4 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 090fc981..4846f765 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 096f4126..0f2da3f8 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.10-SNAPSHOT + 1.11-SNAPSHOT .. From 2d96d1c3864f77dedae261c21a76d6af1cb81b34 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Fri, 13 Dec 2019 21:40:55 +0800 Subject: [PATCH 063/207] [FLINK-15191][table][connectors] Fix connector factories that CREATE TABLE DDL can't work if watermark or computed column is defined This closes #10536 --- .../ElasticsearchUpsertTableSinkBase.java | 3 ++- .../ElasticsearchUpsertTableSinkFactoryBase.java | 15 ++++++++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index e1136421..3c1346f7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -33,6 +33,7 @@ import org.apache.flink.table.sinks.UpsertStreamTableSink; import org.apache.flink.table.typeutils.TypeCheckUtils; import org.apache.flink.table.utils.TableConnectorUtils; +import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; @@ -113,7 +114,7 @@ public ElasticsearchUpsertTableSinkBase( RequestFactory requestFactory) { this.isAppendOnly = isAppendOnly; - this.schema = Preconditions.checkNotNull(schema); + this.schema = TableSchemaUtils.checkNoGeneratedColumns(schema); this.hosts = Preconditions.checkNotNull(hosts); this.index = Preconditions.checkNotNull(index); this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index c9c7f23b..efbbb7e6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -37,6 +37,7 @@ import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.sinks.StreamTableSink; import org.apache.flink.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.types.Row; import org.apache.flink.util.InstantiationUtil; @@ -52,6 +53,11 @@ import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.DescriptorProperties.TABLE_SCHEMA_EXPR; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_DELAY; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE; @@ -142,6 +148,13 @@ public List supportedProperties() { properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_NAME); + // computed column + properties.add(SCHEMA + ".#." + TABLE_SCHEMA_EXPR); + + // watermark + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); // format wildcard properties.add(FORMAT + ".*"); @@ -155,7 +168,7 @@ public StreamTableSink> createStreamTableSink(Map Date: Thu, 9 Jan 2020 10:59:19 +0100 Subject: [PATCH 064/207] [FLINK-14853][core] Implemented MemorySize formatting --- .../ElasticsearchUpsertTableSinkFactoryTestBase.java | 2 +- .../org/apache/flink/table/descriptors/ElasticsearchTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index ea4644b0..d302b88d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -143,7 +143,7 @@ protected Map createTestSinkOptions() { sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_RETRIES, "3"); sinkOptions.put(SinkOption.BULK_FLUSH_INTERVAL, "100"); sinkOptions.put(SinkOption.BULK_FLUSH_MAX_ACTIONS, "1000"); - sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1048576 bytes"); + sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1 mb"); sinkOptions.put(SinkOption.REST_MAX_RETRY_TIMEOUT, "100"); sinkOptions.put(SinkOption.REST_PATH_PREFIX, "/myapp"); return sinkOptions; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java index 85326a63..b994f345 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java @@ -134,7 +134,7 @@ public List> properties() { maximumDesc.put("connector.bulk-flush.backoff.max-retries", "3"); maximumDesc.put("connector.bulk-flush.interval", "100"); maximumDesc.put("connector.bulk-flush.max-actions", "1000"); - maximumDesc.put("connector.bulk-flush.max-size", "12582912 bytes"); + maximumDesc.put("connector.bulk-flush.max-size", "12 mb"); maximumDesc.put("connector.failure-handler", "retry-rejected"); maximumDesc.put("connector.connection-max-retry-timeout", "100"); maximumDesc.put("connector.connection-path-prefix", "/myapp"); From d357fa258e201a0f310b5564fcfc914913be397a Mon Sep 17 00:00:00 2001 From: yanghua Date: Wed, 15 Jan 2020 17:50:15 +0800 Subject: [PATCH 065/207] [FLINK-15558][Connector] Bump Elasticsearch version from 7.3.2 to 7.5.1 for es7 connector --- .../flink-connector-elasticsearch7/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 53 ++++++++++--------- 2 files changed, 28 insertions(+), 27 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index c02037e4..365ee39f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -37,7 +37,7 @@ under the License. - 7.3.2 + 7.5.1 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 7a0ba933..269334ec 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -15,31 +15,32 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpasyncclient:4.1.4 - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 -- org.apache.httpcomponents:httpcore-nio:4.4.11 +- org.apache.httpcomponents:httpcore-nio:4.4.12 - org.apache.logging.log4j:log4j-api:2.11.1 - org.apache.logging.log4j:log4j-to-slf4j:2.11.1 -- org.apache.lucene:lucene-analyzers-common:8.1.0 -- org.apache.lucene:lucene-backward-codecs:8.1.0 -- org.apache.lucene:lucene-core:8.1.0 -- org.apache.lucene:lucene-grouping:8.1.0 -- org.apache.lucene:lucene-highlighter:8.1.0 -- org.apache.lucene:lucene-join:8.1.0 -- org.apache.lucene:lucene-memory:8.1.0 -- org.apache.lucene:lucene-misc:8.1.0 -- org.apache.lucene:lucene-queries:8.1.0 -- org.apache.lucene:lucene-queryparser:8.1.0 -- org.apache.lucene:lucene-sandbox:8.1.0 -- org.apache.lucene:lucene-spatial:8.1.0 -- org.apache.lucene:lucene-spatial-extras:8.1.0 -- org.apache.lucene:lucene-spatial3d:8.1.0 -- org.apache.lucene:lucene-suggest:8.1.0 -- org.elasticsearch:elasticsearch:7.3.2 -- org.elasticsearch:elasticsearch-cli:7.3.2 -- org.elasticsearch:elasticsearch-core:7.3.2 -- org.elasticsearch:elasticsearch-secure-sm:7.3.2 -- org.elasticsearch:elasticsearch-x-content:7.3.2 -- org.elasticsearch.client:elasticsearch-rest-client:7.3.2 -- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.3.2 -- org.elasticsearch.plugin:aggs-matrix-stats-client:7.3.2 -- org.elasticsearch.plugin:parent-join-client:7.3.2 -- org.elasticsearch.plugin:rank-eval-client:7.3.2 +- org.apache.lucene:lucene-analyzers-common:8.3.0 +- org.apache.lucene:lucene-backward-codecs:8.3.0 +- org.apache.lucene:lucene-core:8.3.0 +- org.apache.lucene:lucene-grouping:8.3.0 +- org.apache.lucene:lucene-highlighter:8.3.0 +- org.apache.lucene:lucene-join:8.3.0 +- org.apache.lucene:lucene-memory:8.3.0 +- org.apache.lucene:lucene-misc:8.3.0 +- org.apache.lucene:lucene-queries:8.3.0 +- org.apache.lucene:lucene-queryparser:8.3.0 +- org.apache.lucene:lucene-sandbox:8.3.0 +- org.apache.lucene:lucene-spatial:8.3.0 +- org.apache.lucene:lucene-spatial-extras:8.3.0 +- org.apache.lucene:lucene-spatial3d:8.3.0 +- org.apache.lucene:lucene-suggest:8.3.0 +- org.elasticsearch:elasticsearch:7.5.1 +- org.elasticsearch:elasticsearch-cli:7.5.1 +- org.elasticsearch:elasticsearch-core:7.5.1 +- org.elasticsearch:elasticsearch-secure-sm:7.5.1 +- org.elasticsearch:elasticsearch-x-content:7.5.1 +- org.elasticsearch.client:elasticsearch-rest-client:7.5.1 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.5.1 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.5.1 +- org.elasticsearch.plugin:mapper-extras-client:7.5.1 +- org.elasticsearch.plugin:parent-join-client:7.5.1 +- org.elasticsearch.plugin:rank-eval-client:7.5.1 From 5dd3cb4c56b789d04f774ace318031fb9d283a81 Mon Sep 17 00:00:00 2001 From: Max Kuklinski Date: Thu, 23 Jan 2020 16:53:39 +0100 Subject: [PATCH 066/207] [FLINK-13689] [Connectors/ElasticSearch] Fix thread leak in Elasticsearch connector when cluster is down --- .../ElasticsearchApiCallBridge.java | 12 +++++++- .../elasticsearch/ElasticsearchSinkBase.java | 1 + .../ElasticsearchSinkBaseTest.java | 5 ++++ .../Elasticsearch6ApiCallBridge.java | 29 ++++++++++--------- .../Elasticsearch7ApiCallBridge.java | 29 ++++++++++--------- 5 files changed, 49 insertions(+), 27 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index d3b774c8..7bbbe7b9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -50,7 +50,7 @@ public interface ElasticsearchApiCallBridge extends Ser * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - C createClient(Map clientConfig) throws IOException; + C createClient(Map clientConfig); /** * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. @@ -80,6 +80,16 @@ void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); + /** + * Verify the client connection by making a test request/ping to the Elasticsearch cluster. + * + *

Called by {@link ElasticsearchSinkBase#open(org.apache.flink.configuration.Configuration)} after creating the client. This makes sure the underlying + * client is closed if the connection is not successful and preventing thread leak. + * + * @param client the Elasticsearch client. + */ + void verifyClientConnection(C client) throws IOException; + /** * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 96f44314..c8df2c0f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -297,6 +297,7 @@ public void disableFlushOnCheckpoint() { @Override public void open(Configuration parameters) throws Exception { client = callBridge.createClient(userConfig); + callBridge.verifyClientConnection(client); bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); failureRequestIndexer = new BufferingNoOpRequestIndexer(); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 65ae5ecf..4f960610 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -559,6 +559,11 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { // no need for this in the test cases here } + + @Override + public void verifyClientConnection(Client client) { + // no need for this in the test cases here + } } private static class SimpleSinkFunction implements ElasticsearchSinkFunction { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 782cbbcf..24f97f68 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -68,24 +68,12 @@ public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge clientConfig) throws IOException { + public RestHighLevelClient createClient(Map clientConfig) { RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); restClientFactory.configureRestClientBuilder(builder); RestHighLevelClient rhlClient = new RestHighLevelClient(builder); - if (LOG.isInfoEnabled()) { - LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); - } - - if (!rhlClient.ping()) { - throw new RuntimeException("There are no reachable Elasticsearch nodes!"); - } - - if (LOG.isInfoEnabled()) { - LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); - } - return rhlClient; } @@ -139,4 +127,19 @@ public RequestIndexer createBulkProcessorIndexer( flushOnCheckpoint, numPendingRequestsRef); } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping()) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index c569f3b8..8890eca1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -69,24 +69,12 @@ public class Elasticsearch7ApiCallBridge implements ElasticsearchApiCallBridge clientConfig) throws IOException { + public RestHighLevelClient createClient(Map clientConfig) { RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); restClientFactory.configureRestClientBuilder(builder); RestHighLevelClient rhlClient = new RestHighLevelClient(builder); - if (LOG.isInfoEnabled()) { - LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); - } - - if (!rhlClient.ping(RequestOptions.DEFAULT)) { - throw new RuntimeException("There are no reachable Elasticsearch nodes!"); - } - - if (LOG.isInfoEnabled()) { - LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); - } - return rhlClient; } @@ -140,4 +128,19 @@ public RequestIndexer createBulkProcessorIndexer( flushOnCheckpoint, numPendingRequestsRef); } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } } From 0bff02d53ce89567286d746ae4b0d2bb30e3d9ff Mon Sep 17 00:00:00 2001 From: Thomas Weise Date: Mon, 3 Feb 2020 15:13:19 -0800 Subject: [PATCH 067/207] [FLINK-15868][kinesis] Resolve version conflict between jackson-core and jackson-dataformat-cbor --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 20dec329..0553ffd1 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 - commons-codec:commons-codec:1.10 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 269334ec..9ee1428c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.11 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 - commons-codec:commons-codec:1.10 From 139f471acf96c4c7885a91168009c113d2e64873 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 4 Feb 2020 16:45:15 +0100 Subject: [PATCH 068/207] [FLINK-15868] Pin jackson-dataformat-smile dependency to 2.10.1 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 0553ffd1..e4072986 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -8,7 +8,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 9ee1428c..435d9b2a 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -8,7 +8,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 From 0aefb86a1cb0fba140f8e09df326f3529efe1626 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Tue, 4 Feb 2020 16:46:48 +0100 Subject: [PATCH 069/207] [FLINK-15868] Pin jackson-dataformat-yaml dependency to 2.10.1 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index e4072986..b161f149 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -9,7 +9,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.10 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 435d9b2a..4de5a181 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -9,7 +9,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.8.11 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.4 From 48def607dd4ea7296f539772acb8e33549b072c2 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 4 Feb 2020 16:51:25 +0100 Subject: [PATCH 070/207] [hotfix][es][tests] Remove unused variable Subsumed by #getUserConfig() --- .../elasticsearch/ElasticsearchSinkTestBase.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 819ffba5..e588cc8e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -112,10 +112,6 @@ public void runElasticsearchSinkTest() throws Exception { * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code null}. */ public void runNullAddressesTest() throws Exception { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", CLUSTER_NAME); - try { createElasticsearchSink( 1, @@ -134,10 +130,6 @@ public void runNullAddressesTest() throws Exception { * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. */ public void runEmptyAddressesTest() throws Exception { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", CLUSTER_NAME); - try { createElasticsearchSink( 1, @@ -160,10 +152,6 @@ public void runInvalidElasticsearchClusterTest() throws Exception { DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "invalid-cluster-name"); - source.addSink(createElasticsearchSinkForNode( 1, "invalid-cluster-name", From c377cf05616a19c8be1c2ff9c3fb75dd5daccbe7 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 4 Feb 2020 16:53:42 +0100 Subject: [PATCH 071/207] [hotfix][es][tests] Refactor utils --- .../ElasticsearchSinkTestBase.java | 15 ++++++--- .../testutils/SourceSinkDataTestKit.java | 32 ++++++++++++++----- 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index e588cc8e..fa72a0f4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Function; import static org.junit.Assert.fail; @@ -85,9 +86,13 @@ public static void shutdown() throws Exception { } /** - * Tests that the Elasticsearch sink works properly. + * Tests that the Elasticsearch sink works properly with json. */ public void runElasticsearchSinkTest() throws Exception { + runElasticSearchSinkTest(SourceSinkDataTestKit::getJsonSinkFunction); + } + + private void runElasticSearchSinkTest(Function>> functionFactory) throws Exception { final String index = "elasticsearch-sink-test-index"; final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -97,7 +102,7 @@ public void runElasticsearchSinkTest() throws Exception { source.addSink(createElasticsearchSinkForEmbeddedNode( 1, CLUSTER_NAME, - new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + functionFactory.apply(index))); env.execute("Elasticsearch Sink Test"); @@ -117,7 +122,7 @@ public void runNullAddressesTest() throws Exception { 1, CLUSTER_NAME, null, - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + SourceSinkDataTestKit.getJsonSinkFunction("test")); } catch (IllegalArgumentException | NullPointerException expectedException) { // test passes return; @@ -135,7 +140,7 @@ public void runEmptyAddressesTest() throws Exception { 1, CLUSTER_NAME, Collections.emptyList(), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + SourceSinkDataTestKit.getJsonSinkFunction("test")); } catch (IllegalArgumentException expectedException) { // test passes return; @@ -155,7 +160,7 @@ public void runInvalidElasticsearchClusterTest() throws Exception { source.addSink(createElasticsearchSinkForNode( 1, "invalid-cluster-name", - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"), + SourceSinkDataTestKit.getJsonSinkFunction("test"), "123.123.123.123")); // incorrect ip address try { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 32498c6c..95aba9f7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -27,8 +27,12 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.junit.Assert; +import java.io.IOException; +import java.io.Serializable; import java.util.HashMap; import java.util.Map; @@ -66,28 +70,35 @@ public void cancel() { } } - /** - * A {@link ElasticsearchSinkFunction} that indexes each element it receives to a specified Elasticsearch index. - */ - public static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { + public static ElasticsearchSinkFunction> getJsonSinkFunction(String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::jsonBuilder); + } + + private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { private static final long serialVersionUID = 1L; private final String index; + private final XContentBuilderProvider contentBuilderProvider; /** * Create the sink function, specifying a target Elasticsearch index. * * @param index Name of the target Elasticsearch index. */ - public TestElasticsearchSinkFunction(String index) { + public TestElasticsearchSinkFunction(String index, XContentBuilderProvider contentBuilderProvider) { this.index = index; + this.contentBuilderProvider = contentBuilderProvider; } public IndexRequest createIndexRequest(Tuple2 element) { - Map json = new HashMap<>(); - json.put(DATA_FIELD_NAME, element.f1); + Map document = new HashMap<>(); + document.put(DATA_FIELD_NAME, element.f1); - return new IndexRequest(index, TYPE_NAME, element.f0.toString()).source(json); + try { + return new IndexRequest(index, TYPE_NAME, element.f0.toString()).source(contentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override @@ -110,4 +121,9 @@ public static void verifyProducedSinkData(Client client, String index) { } } + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } + } From bc5856e7435e30ef9a0ca528ec4fcf2dcdf8df7d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 4 Feb 2020 16:55:03 +0100 Subject: [PATCH 072/207] [FLINK-15868][es] Add tests for different formats --- .../ElasticsearchSinkTestBase.java | 21 +++++++++++++++++++ .../testutils/SourceSinkDataTestKit.java | 12 +++++++++++ .../ElasticsearchSinkITCase.java | 5 +++++ .../ElasticsearchSinkITCase.java | 5 +++++ 4 files changed, 43 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index fa72a0f4..8d65e428 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -92,6 +92,27 @@ public void runElasticsearchSinkTest() throws Exception { runElasticSearchSinkTest(SourceSinkDataTestKit::getJsonSinkFunction); } + /** + * Tests that the Elasticsearch sink works properly with cbor. + */ + public void runElasticsearchSinkCborTest() throws Exception { + runElasticSearchSinkTest(SourceSinkDataTestKit::getCborSinkFunction); + } + + /** + * Tests that the Elasticsearch sink works properly with smile. + */ + public void runElasticsearchSinkSmileTest() throws Exception { + runElasticSearchSinkTest(SourceSinkDataTestKit::getSmileSinkFunction); + } + + /** + * Tests that the Elasticsearch sink works properly with yaml. + */ + public void runElasticsearchSinkYamlTest() throws Exception { + runElasticSearchSinkTest(SourceSinkDataTestKit::getYamlSinkFunction); + } + private void runElasticSearchSinkTest(Function>> functionFactory) throws Exception { final String index = "elasticsearch-sink-test-index"; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 95aba9f7..7bcc7001 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -70,10 +70,22 @@ public void cancel() { } } + public static ElasticsearchSinkFunction> getCborSinkFunction(String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::cborBuilder); + } + public static ElasticsearchSinkFunction> getJsonSinkFunction(String index) { return new TestElasticsearchSinkFunction(index, XContentFactory::jsonBuilder); } + public static ElasticsearchSinkFunction> getSmileSinkFunction(String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::smileBuilder); + } + + public static ElasticsearchSinkFunction> getYamlSinkFunction(String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::yamlBuilder); + } + private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { private static final long serialVersionUID = 1L; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index a6f01258..f2db0625 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -44,6 +44,11 @@ public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); } + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + @Test public void testNullAddresses() throws Exception { runNullAddressesTest(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index c8de4473..65ad40a0 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -44,6 +44,11 @@ public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); } + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + @Test public void testNullAddresses() throws Exception { runNullAddressesTest(); From eef304eadf8dce436d170aaae3036ca355b2b366 Mon Sep 17 00:00:00 2001 From: Victor Wong Date: Wed, 12 Feb 2020 20:34:50 +0800 Subject: [PATCH 073/207] [FLINK-15992][kafka][es] Use thread context classloader when finding TableFormatFactory (#11064) --- .../elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index efbbb7e6..2b4cb837 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -243,8 +243,7 @@ private SerializationSchema getSerializationSchema(Map prop @SuppressWarnings("unchecked") final SerializationSchemaFactory formatFactory = TableFactoryService.find( SerializationSchemaFactory.class, - properties, - this.getClass().getClassLoader()); + properties); return formatFactory.createSerializationSchema(properties); } From a21cc0ffacc29e606aedb0155eb8cc333d5fc5f4 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 14 Feb 2020 09:42:41 +0100 Subject: [PATCH 074/207] [FLINK-16046][es] Drop Elasticsearch 2 connector --- .../pom.xml | 39 ++++++++++++++++++- .../ElasticsearchSinkBaseTest.java | 4 +- .../flink-connector-elasticsearch6/pom.xml | 6 +++ 3 files changed, 46 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 56d4b6b5..1e9722b7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 2.3.5 + 5.1.2 @@ -139,10 +139,47 @@ under the License. test + + + + org.apache.logging.log4j + log4j-api + 2.7 + test + + + + org.apache.logging.log4j + log4j-core + 2.7 + test + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.logging.log4j:log4j-to-slf4j + + + + org.apache.maven.plugins maven-jar-plugin diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 4f960610..f0e401ef 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -26,7 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; @@ -510,7 +510,7 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { if (mockItemFailure == null) { // the mock response for the item is success - mockResponses[i] = new BulkItemResponse(i, "opType", mock(ActionWriteResponse.class)); + mockResponses[i] = new BulkItemResponse(i, "opType", mock(DocWriteResponse.class)); } else { // the mock response for the item is failure mockResponses[i] = new BulkItemResponse(i, "opType", new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index fa4799fa..2a594145 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -148,6 +148,12 @@ under the License. embedded Elasticsearch nodes used in tests to run correctly. --> + + org.apache.logging.log4j + log4j-api + 2.9.1 + + org.apache.logging.log4j log4j-core From 132d07e8519e027ff2f70eeae75d816a19c41b1a Mon Sep 17 00:00:00 2001 From: Benchao Li Date: Tue, 18 Feb 2020 14:54:41 +0800 Subject: [PATCH 075/207] [FLINK-15988][json] Make JsonRowSerializationSchema's constructor private (#11080) --- .../ElasticsearchUpsertTableSinkFactoryTestBase.java | 4 ++-- .../Elasticsearch6UpsertTableSinkFactoryTest.java | 4 ++-- .../Elasticsearch7UpsertTableSinkFactoryTest.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index d302b88d..d82b2df1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -77,7 +77,7 @@ public void testTableSink() { DOC_TYPE, KEY_DELIMITER, KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), createTestSinkOptions()); @@ -103,7 +103,7 @@ public void testTableSinkWithLegacyProperties() { DOC_TYPE, KEY_DELIMITER, KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), createTestSinkOptions()); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index a9f08149..257634d1 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -70,7 +70,7 @@ public void testBuilder() { DOC_TYPE, KEY_DELIMITER, KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), createTestSinkOptions()); @@ -88,7 +88,7 @@ public void testBuilder() { DOC_TYPE, KEY_DELIMITER, KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, Elasticsearch6UpsertTableSink.UPDATE_REQUEST_FACTORY, new int[0])); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index 1b984523..5837733b 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -69,7 +69,7 @@ public void testBuilder() { ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), createTestSinkOptions()); @@ -87,7 +87,7 @@ public void testBuilder() { ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, - new JsonRowSerializationSchema(schema.toRowType()), + JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, new int[0])); From 8bc4e2f6d360324ae30d119352e15f2883c8f869 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 21 Feb 2020 20:23:32 +0800 Subject: [PATCH 076/207] [hotfix] Revert "[FLINK-15992][kafka][es] Use thread context classloader when finding TableFormatFactory (#11064)" This reverts commit a93ee07d --- .../elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index 2b4cb837..efbbb7e6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -243,7 +243,8 @@ private SerializationSchema getSerializationSchema(Map prop @SuppressWarnings("unchecked") final SerializationSchemaFactory formatFactory = TableFactoryService.find( SerializationSchemaFactory.class, - properties); + properties, + this.getClass().getClassLoader()); return formatFactory.createSerializationSchema(properties); } From 7bf30d26cab5e28a37c2ab626d5bd1fda091b91a Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 24 Jan 2020 13:49:28 +0100 Subject: [PATCH 077/207] [FLINK-15672][build] Migrate to log4j2 --- .../pom.xml | 22 +------- .../flink-connector-elasticsearch6/pom.xml | 53 +------------------ .../ElasticsearchSinkITCase.java | 4 -- .../flink-connector-elasticsearch7/pom.xml | 51 ------------------ .../ElasticsearchSinkITCase.java | 4 -- .../src/main/resources/META-INF/NOTICE | 2 - .../src/main/resources/META-INF/NOTICE | 2 - 7 files changed, 2 insertions(+), 136 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 1e9722b7..113a4756 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -147,14 +147,12 @@ under the License. org.apache.logging.log4j log4j-api - 2.7 - test + provided org.apache.logging.log4j log4j-core - 2.7 test @@ -162,24 +160,6 @@ under the License. - - - org.apache.maven.plugins - maven-surefire-plugin - - - org.apache.logging.log4j:log4j-to-slf4j - - - - org.apache.maven.plugins maven-jar-plugin diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 2a594145..25a9f5a7 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -71,19 +71,6 @@ under the License. ${elasticsearch.version} - - - - org.apache.logging.log4j - log4j-to-slf4j - 2.9.1 - - @@ -143,22 +130,10 @@ under the License. test - - org.apache.logging.log4j log4j-api - 2.9.1 - - - - org.apache.logging.log4j - log4j-core - 2.9.1 - test + provided @@ -179,30 +154,4 @@ under the License. - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - true - - - org.apache.logging.log4j:log4j-to-slf4j - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index f2db0625..27903df8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -32,10 +32,6 @@ /** * IT cases for the {@link ElasticsearchSink}. - * - *

The Elasticsearch ITCases for 6.x CANNOT be executed in the IDE directly, since it is required that the - * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded - * node used in the tests to work properly. */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 365ee39f..5f25f46c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -71,19 +71,6 @@ under the License. ${elasticsearch.version} - - - - org.apache.logging.log4j - log4j-to-slf4j - 2.11.1 - - @@ -143,18 +130,6 @@ under the License. test - - - - org.apache.logging.log4j - log4j-core - 2.9.1 - test - - org.apache.flink @@ -173,30 +148,4 @@ under the License. - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - true - - - org.apache.logging.log4j:log4j-to-slf4j - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 65ad40a0..66f1eabc 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -32,10 +32,6 @@ /** * IT cases for the {@link ElasticsearchSink}. - * - *

The Elasticsearch ITCases for 7.x CANNOT be executed in the IDE directly, since it is required that the - * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded - * node used in the tests to work properly. */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index b161f149..43c60595 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -16,8 +16,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 - org.apache.httpcomponents:httpcore-nio:4.4.5 -- org.apache.logging.log4j:log4j-api:2.9.1 -- org.apache.logging.log4j:log4j-to-slf4j:2.9.1 - org.apache.lucene:lucene-analyzers-common:7.3.1 - org.apache.lucene:lucene-backward-codecs:7.3.1 - org.apache.lucene:lucene-core:7.3.1 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 4de5a181..0a27b7bc 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -16,8 +16,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpclient:4.5.3 - org.apache.httpcomponents:httpcore:4.4.6 - org.apache.httpcomponents:httpcore-nio:4.4.12 -- org.apache.logging.log4j:log4j-api:2.11.1 -- org.apache.logging.log4j:log4j-to-slf4j:2.11.1 - org.apache.lucene:lucene-analyzers-common:8.3.0 - org.apache.lucene:lucene-backward-codecs:8.3.0 - org.apache.lucene:lucene-core:8.3.0 From 468663f2a0137d89a101b145b8b9a69cd65f2d0b Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 24 Jan 2020 15:21:40 +0100 Subject: [PATCH 078/207] [FLINK-15672][build][tests] Migrate test log4j configurations to log4j2 --- .../src/test/resources/log4j2-test.properties} | 14 +++++++++----- .../src/test/resources/log4j2-test.properties} | 14 +++++++++----- .../src/test/resources/log4j2-test.properties} | 17 +++++++++-------- 3 files changed, 27 insertions(+), 18 deletions(-) rename flink-connectors/{flink-connector-elasticsearch6/src/test/resources/log4j-test.properties => flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties} (70%) rename flink-connectors/{flink-connector-elasticsearch7/src/test/resources/log4j-test.properties => flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties} (70%) rename flink-connectors/{flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties => flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties} (68%) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties similarity index 70% rename from flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties rename to flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties index fcd86546..835c2ec9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties @@ -16,9 +16,13 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, testlogger +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger -log4j.appender.testlogger=org.apache.log4j.ConsoleAppender -log4j.appender.testlogger.target=System.err -log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties similarity index 70% rename from flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties rename to flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties index fcd86546..835c2ec9 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties @@ -16,9 +16,13 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, testlogger +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger -log4j.appender.testlogger=org.apache.log4j.ConsoleAppender -log4j.appender.testlogger.target=System.err -log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties similarity index 68% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties rename to flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties index 20551848..835c2ec9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties @@ -16,12 +16,13 @@ # limitations under the License. ################################################################################ -log4j.rootLogger=INFO, testlogger +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger -log4j.appender.testlogger=org.apache.log4j.ConsoleAppender -log4j.appender.testlogger.target=System.err -log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n - -# suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n From 427ce9070c0e0634a35ad52077622bb2143dd443 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 20 Feb 2020 12:10:42 +0100 Subject: [PATCH 079/207] [FLINK-16186][es][tests] Reduce connect timeout to 5 seconds --- .../connectors/elasticsearch/ElasticsearchSinkTestBase.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 8d65e428..8b568962 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -187,6 +187,7 @@ public void runInvalidElasticsearchClusterTest() throws Exception { try { env.execute("Elasticsearch Sink Test"); } catch (JobExecutionException expectedException) { + // every ES version throws a different exception in case of timeouts, so don't bother asserting on the exception // test passes return; } @@ -201,6 +202,7 @@ protected Map createUserConfig(int bulkFlushMaxActions, String c Map userConfig = new HashMap<>(); userConfig.put("cluster.name", clusterName); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(bulkFlushMaxActions)); + userConfig.put("transport.tcp.connect_timeout", "5s"); return userConfig; } From 21523c290e2b660cfdb3473be2c447335716db22 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 26 Feb 2020 10:58:05 +0100 Subject: [PATCH 080/207] [FLINK-16287][es][build] Remove Log4j2 relocation --- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 4846f765..f0a7a643 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -132,10 +132,6 @@ under the License. org.elasticsearch org.apache.flink.elasticsearch6.shaded.org.elasticsearch - - org.apache.logging - org.apache.flink.elasticsearch6.shaded.org.apache.logging - com.fasterxml.jackson org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson From bb73a363de229ef34449d109b818a987c359f9f4 Mon Sep 17 00:00:00 2001 From: godfrey he Date: Wed, 4 Mar 2020 15:52:37 +0800 Subject: [PATCH 081/207] [FLINK-16362][table] Remove deprecated `emitDataStream` method in StreamTableSink This closes #11279 --- .../elasticsearch/ElasticsearchUpsertTableSinkBase.java | 5 ----- .../Elasticsearch6UpsertTableSinkFactoryTest.java | 2 +- .../Elasticsearch7UpsertTableSinkFactoryTest.java | 2 +- 3 files changed, 2 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 3c1346f7..7ee5d924 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -191,11 +191,6 @@ public DataStreamSink consumeDataStream(DataStream> data .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); } - @Override - public void emitDataStream(DataStream> dataStream) { - consumeDataStream(dataStream); - } - @Override public TypeInformation> getOutputType() { return Types.TUPLE(Types.BOOLEAN, getRecordType()); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 257634d1..021bbdd0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -79,7 +79,7 @@ public void testBuilder() { new StreamExecutionEnvironmentMock(), Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - testSink.emitDataStream(dataStreamMock); + testSink.consumeDataStream(dataStreamMock); final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index 5837733b..f3493b34 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -78,7 +78,7 @@ public void testBuilder() { new StreamExecutionEnvironmentMock(), Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - testSink.emitDataStream(dataStreamMock); + testSink.consumeDataStream(dataStreamMock); final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( Collections.singletonList(new HttpHost(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), From 2c5ae1e2fce5fd056068f9cd73b376c413d4ce16 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Tue, 18 Feb 2020 13:43:23 +0100 Subject: [PATCH 082/207] [FLINK-11720][connectors] Bump ElasticSearch5 to 5.3.3 The ES5 connector has caused numerous issues in end to end and integration tests (on CI and during release test). The NOTICE file has been updated according to this maven shade output: [INFO] --- maven-shade-plugin:3.1.1:shade (shade-flink) @ flink-connector-elasticsearch5_2.11 --- [INFO] Including org.apache.flink:flink-connector-elasticsearch-base_2.11:jar:1.11-SNAPSHOT in the shaded jar. [INFO] Including org.elasticsearch.client:transport:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch:elasticsearch:jar:5.3.3 in the shaded jar. [INFO] Including org.apache.lucene:lucene-core:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-analyzers-common:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-backward-codecs:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-grouping:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-highlighter:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-join:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-memory:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-misc:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-queries:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-queryparser:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-sandbox:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial-extras:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-spatial3d:jar:6.4.2 in the shaded jar. [INFO] Including org.apache.lucene:lucene-suggest:jar:6.4.2 in the shaded jar. [INFO] Including org.elasticsearch:securesm:jar:1.1 in the shaded jar. [INFO] Including net.sf.jopt-simple:jopt-simple:jar:5.0.2 in the shaded jar. [INFO] Including com.carrotsearch:hppc:jar:0.7.1 in the shaded jar. [INFO] Including joda-time:joda-time:jar:2.5 in the shaded jar. [INFO] Including org.yaml:snakeyaml:jar:1.25 in the shaded jar. [INFO] Including com.fasterxml.jackson.core:jackson-core:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-smile:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:jar:2.10.1 in the shaded jar. [INFO] Including com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:jar:2.10.1 in the shaded jar. [INFO] Including com.tdunning:t-digest:jar:3.0 in the shaded jar. [INFO] Including org.hdrhistogram:HdrHistogram:jar:2.1.6 in the shaded jar. [INFO] Including net.java.dev.jna:jna:jar:4.2.2 in the shaded jar. [INFO] Including org.elasticsearch.plugin:transport-netty3-client:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:transport-netty4-client:jar:5.3.3 in the shaded jar. [INFO] Including io.netty:netty-buffer:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-codec-http:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-common:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-handler:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-resolver:jar:4.1.7.Final in the shaded jar. [INFO] Including io.netty:netty-transport:jar:4.1.7.Final in the shaded jar. [INFO] Including org.elasticsearch.plugin:reindex-client:jar:5.3.3 in the shaded jar. [INFO] Including org.elasticsearch.client:rest:jar:5.3.3 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpclient:jar:4.5.3 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpcore:jar:4.4.6 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpasyncclient:jar:4.1.2 in the shaded jar. [INFO] Including org.apache.httpcomponents:httpcore-nio:jar:4.4.5 in the shaded jar. [INFO] Including commons-codec:commons-codec:jar:1.10 in the shaded jar. [INFO] Including commons-logging:commons-logging:jar:1.1.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:lang-mustache-client:jar:5.3.3 in the shaded jar. [INFO] Including com.github.spullara.mustache.java:compiler:jar:0.9.3 in the shaded jar. [INFO] Including org.elasticsearch.plugin:percolator-client:jar:5.3.3 in the shaded jar. [INFO] Including io.netty:netty:jar:3.10.6.Final in the shaded jar. [INFO] Including org.apache.flink:force-shading:jar:1.11-SNAPSHOT in the shaded jar. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [WARNING] Discovered module-info.class. Shading will break its strong encapsulation. [INFO] Replacing original artifact with shaded artifact. [INFO] Replacing /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/flink-connector-elasticsearch5_2.11-1.11-SNAPSHOT.jar with /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/flink-connector-elasticsearch5_2.11-1.11-SNAPSHOT-shaded.jar [INFO] Dependency-reduced POM written at: /Users/robert/Projects/flink/flink-connectors/flink-connector-elasticsearch5/target/dependency-reduced-pom.xml [INFO] [INFO] --- maven-surefire-plugin:2.22.1:test (integration-tests) @ flink-connector-elasticsearch5_2.11 --- --- .../pom.xml | 2 +- .../elasticsearch/ElasticsearchSinkBase.java | 24 +++++++++++++++---- .../ElasticsearchSinkBaseTest.java | 11 +++++---- 3 files changed, 27 insertions(+), 10 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 113a4756..f5c4252b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 5.1.2 + 5.3.3 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index c8df2c0f..c285112d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -29,6 +29,7 @@ import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; @@ -389,6 +390,7 @@ private void checkAsyncErrorsAndRequests() { } private class BulkProcessorListener implements BulkProcessor.Listener { + @Override public void beforeBulk(long executionId, BulkRequest request) { } @@ -398,6 +400,7 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon BulkItemResponse itemResponse; Throwable failure; RestStatus restStatus; + DocWriteRequest actionRequest; try { for (int i = 0; i < response.getItems().length; i++) { @@ -407,10 +410,19 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); restStatus = itemResponse.getFailure().getStatus(); + actionRequest = request.requests().get(i); if (restStatus == null) { - failureHandler.onFailure(request.requests().get(i), failure, -1, failureRequestIndexer); + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) actionRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } else { - failureHandler.onFailure(request.requests().get(i), failure, restStatus.getStatus(), failureRequestIndexer); + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) actionRequest, failure, restStatus.getStatus(), failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } } } @@ -431,8 +443,12 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); try { - for (ActionRequest action : request.requests()) { - failureHandler.onFailure(action, failure, -1, failureRequestIndexer); + for (DocWriteRequest writeRequest : request.requests()) { + if (writeRequest instanceof ActionRequest) { + failureHandler.onFailure((ActionRequest) writeRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); + } } } catch (Throwable t) { // fail the sink and skip the rest of the items diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index f0e401ef..0648ed91 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; @@ -419,7 +420,7 @@ private static class DummyElasticsearchSink extends ElasticsearchSinkBase mockItemFailuresList; + private List mockItemFailuresList; private Throwable nextBulkFailure; public DummyElasticsearchSink( @@ -454,7 +455,7 @@ public void continueFlush() { *

The list is used with corresponding order to the requests in the bulk, i.e. the first * request uses the response at index 0, the second requests uses the response at index 1, etc. */ - public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { + public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { this.mockItemFailuresList = mockItemFailuresList; } @@ -506,14 +507,14 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { if (nextBulkFailure == null) { BulkItemResponse[] mockResponses = new BulkItemResponse[currentBulkRequest.requests().size()]; for (int i = 0; i < currentBulkRequest.requests().size(); i++) { - Throwable mockItemFailure = mockItemFailuresList.get(i); + Exception mockItemFailure = mockItemFailuresList.get(i); if (mockItemFailure == null) { // the mock response for the item is success - mockResponses[i] = new BulkItemResponse(i, "opType", mock(DocWriteResponse.class)); + mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, mock(DocWriteResponse.class)); } else { // the mock response for the item is failure - mockResponses[i] = new BulkItemResponse(i, "opType", new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); + mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); } } From 11fa310f42cb1285efcea37f086030040bcb7242 Mon Sep 17 00:00:00 2001 From: Zhijiang Date: Mon, 16 Mar 2020 13:31:51 +0800 Subject: [PATCH 083/207] [FLINK-16454][build] Update the copyright with 2020 year in NOTICE files --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 43c60595..542a1f6c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch6 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 0a27b7bc..9f68ec03 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch7 -Copyright 2014-2019 The Apache Software Foundation +Copyright 2014-2020 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From c38b700771d6bf2f15c500f58a0a8838b4e2caa9 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Wed, 25 Mar 2020 10:13:41 +0800 Subject: [PATCH 084/207] [FLINK-16170][elasticsearch] Fix SearchTemplateRequest ClassNotFoundException when using flink-sql-connector-elasticsearch7 We shouldn't `exclude org.elasticsearch:elasticsearch-geo` and `org.elasticsearch.plugin:lang-mustache-client` when shading. This closes #11396 --- .../flink-sql-connector-elasticsearch7/pom.xml | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 0f2da3f8..d1e289dc 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -63,14 +63,10 @@ under the License. - com.carrotsearch:hppc com.tdunning:t-digest joda-time:joda-time net.sf.jopt-simple:jopt-simple org.elasticsearch:jna - org.elasticsearch:elasticsearch-geo - org.elasticsearch.plugin:lang-mustache-client - com.github.spullara.mustache.java:compiler org.hdrhistogram:HdrHistogram org.yaml:snakeyaml @@ -135,14 +131,18 @@ under the License. org.elasticsearch org.apache.flink.elasticsearch7.shaded.org.elasticsearch - - org.apache.logging - org.apache.flink.elasticsearch7.shaded.org.apache.logging - com.fasterxml.jackson org.apache.flink.elasticsearch7.shaded.com.fasterxml.jackson + + com.carrotsearch.hppc + org.apache.flink.elasticsearch7.shaded.com.carrotsearch.hppc + + + com.github.mustachejava + org.apache.flink.elasticsearch7.shaded.com.github.mustachejava + From f6bc79d6f969274c292ea1e4f6886efde4fae397 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Sat, 18 Apr 2020 22:42:48 +0800 Subject: [PATCH 085/207] [FLINK-15400][elasticsearch] Support dynamic index for Elasticsearch sink connector This closes #11466 --- .../elasticsearch/ElasticsearchSinkBase.java | 2 +- .../ElasticsearchSinkFunction.java | 5 + .../ElasticsearchUpsertTableSinkBase.java | 35 ++- .../index/AbstractTimeIndexGenerator.java | 40 +++ .../elasticsearch/index/IndexGenerator.java | 41 +++ .../index/IndexGeneratorBase.java | 51 ++++ .../index/IndexGeneratorFactory.java | 279 ++++++++++++++++++ .../index/StaticIndexGenerator.java | 35 +++ ...csearchUpsertTableSinkFactoryTestBase.java | 11 +- .../index/IndexGeneratorTest.java | 246 +++++++++++++++ ...sticsearch6UpsertTableSinkFactoryTest.java | 9 +- ...sticsearch7UpsertTableSinkFactoryTest.java | 27 +- 12 files changed, 749 insertions(+), 32 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index c285112d..91beaf06 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -212,7 +212,6 @@ public ElasticsearchSinkBase( this.callBridge = checkNotNull(callBridge); this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); this.failureHandler = checkNotNull(failureHandler); - // we eagerly check if the user-provided sink function and failure handler is serializable; // otherwise, if they aren't serializable, users will merely get a non-informative error message // "ElasticsearchSinkBase is not serializable" @@ -302,6 +301,7 @@ public void open(Configuration parameters) throws Exception { bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); failureRequestIndexer = new BufferingNoOpRequestIndexer(); + elasticsearchSinkFunction.open(); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index 1b5ce1e0..8ff6babf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -60,6 +60,11 @@ @PublicEvolving public interface ElasticsearchSinkFunction extends Serializable, Function { + /** + * Initialization method for the function. It is called once before the actual working process methods. + */ + default void open() {} + /** * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. * The produced requests should be added to the provided {@link RequestIndexer}. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 7ee5d924..defc804b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -27,6 +27,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.sinks.TableSink; @@ -173,7 +175,7 @@ public TypeInformation getRecordType() { public DataStreamSink consumeDataStream(DataStream> dataStream) { final ElasticsearchUpsertSinkFunction upsertFunction = new ElasticsearchUpsertSinkFunction( - index, + IndexGeneratorFactory.createIndexGenerator(index, schema), docType, keyDelimiter, keyNullLiteral, @@ -414,7 +416,7 @@ DeleteRequest createDeleteRequest( */ public static class ElasticsearchUpsertSinkFunction implements ElasticsearchSinkFunction> { - private final String index; + private final IndexGenerator indexGenerator; private final String docType; private final String keyDelimiter; private final String keyNullLiteral; @@ -424,7 +426,7 @@ public static class ElasticsearchUpsertSinkFunction implements ElasticsearchSink private final int[] keyFieldIndices; public ElasticsearchUpsertSinkFunction( - String index, + IndexGenerator indexGenerator, String docType, String keyDelimiter, String keyNullLiteral, @@ -433,7 +435,7 @@ public ElasticsearchUpsertSinkFunction( RequestFactory requestFactory, int[] keyFieldIndices) { - this.index = Preconditions.checkNotNull(index); + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); this.docType = Preconditions.checkNotNull(docType); this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); this.serializationSchema = Preconditions.checkNotNull(serializationSchema); @@ -443,20 +445,27 @@ public ElasticsearchUpsertSinkFunction( this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); } + @Override + public void open() { + indexGenerator.open(); + } + @Override public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + + final String formattedIndex = indexGenerator.generate(element.f1); if (element.f0) { - processUpsert(element.f1, indexer); + processUpsert(element.f1, indexer, formattedIndex); } else { - processDelete(element.f1, indexer); + processDelete(element.f1, indexer, formattedIndex); } } - private void processUpsert(Row row, RequestIndexer indexer) { + private void processUpsert(Row row, RequestIndexer indexer, String formattedIndex) { final byte[] document = serializationSchema.serialize(row); if (keyFieldIndices.length == 0) { final IndexRequest indexRequest = requestFactory.createIndexRequest( - index, + formattedIndex, docType, contentType, document); @@ -464,7 +473,7 @@ private void processUpsert(Row row, RequestIndexer indexer) { } else { final String key = createKey(row); final UpdateRequest updateRequest = requestFactory.createUpdateRequest( - index, + formattedIndex, docType, key, contentType, @@ -473,10 +482,10 @@ private void processUpsert(Row row, RequestIndexer indexer) { } } - private void processDelete(Row row, RequestIndexer indexer) { + private void processDelete(Row row, RequestIndexer indexer, String formattedIndex) { final String key = createKey(row); final DeleteRequest deleteRequest = requestFactory.createDeleteRequest( - index, + formattedIndex, docType, key); indexer.add(deleteRequest); @@ -508,7 +517,7 @@ public boolean equals(Object o) { return false; } ElasticsearchUpsertSinkFunction that = (ElasticsearchUpsertSinkFunction) o; - return Objects.equals(index, that.index) && + return Objects.equals(indexGenerator, that.indexGenerator) && Objects.equals(docType, that.docType) && Objects.equals(keyDelimiter, that.keyDelimiter) && Objects.equals(keyNullLiteral, that.keyNullLiteral) && @@ -521,7 +530,7 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = Objects.hash( - index, + indexGenerator, docType, keyDelimiter, keyNullLiteral, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000..52141450 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java @@ -0,0 +1,40 @@ +/* + * 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.connectors.elasticsearch.index; + +import java.time.format.DateTimeFormatter; + +/** + * Abstract class for time related {@link IndexGenerator}. + */ +public abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java new file mode 100644 index 00000000..d45fc1be --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java @@ -0,0 +1,41 @@ +/* + * 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.connectors.elasticsearch.index; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** + * This interface is responsible to generate index name from given {@link Row} record. + */ +@Internal +public interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link #generate(Row)} is called. + */ + default void open() {} + + /** + * Generate index name according the the given row. + */ + String generate(Row row); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java new file mode 100644 index 00000000..00d95e9e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.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 org.apache.flink.streaming.connectors.elasticsearch.index; + +import java.util.Objects; + +/** + * Base class for {@link IndexGenerator}. + */ +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java new file mode 100644 index 00000000..9f84051d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java @@ -0,0 +1,279 @@ +/* + * 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.connectors.elasticsearch.index; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. 'myusers', + * all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in the + * record to dynamically generate a target index. You can also use '{field_name|date_format_string}' to + * convert a field value of TIMESTAMP/DATE/TIME type into the format specified by date_format_string. The + * date_format_string is compatible with {@link java.text.SimpleDateFormat}. For example, if the option + * value is 'myusers_{log_ts|yyyy-MM-dd}', then a record with log_ts field value 2020-03-27 12:25:55 will + * be written into "myusers-2020-03-27" index. + */ +@Internal +public class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator(index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final TypeInformation indexFieldType = TypeConversions.fromDataTypeToLegacyInfo(fieldTypes[indexFieldPos]); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldType); + + // time extract dynamic index pattern + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldType); + // DataTypes.SQL_TIMESTAMP + if (indexFieldType == Types.LOCAL_DATE_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalDateTime indexField = (LocalDateTime) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } + else if (indexFieldType == Types.SQL_TIMESTAMP) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Timestamp indexField = (Timestamp) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.toLocalDateTime().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } + // DataTypes.SQL_DATE + else if (indexFieldType == Types.LOCAL_DATE) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalDate indexField = (LocalDate) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else if (indexFieldType == Types.SQL_DATE) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Date indexField = (Date) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.toLocalDate().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } // DataTypes.TIME + else if (indexFieldType == Types.LOCAL_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalTime indexField = (LocalTime) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else if (indexFieldType == Types.SQL_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Time indexField = (Time) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.toLocalTime().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else { + throw new TableException(String.format("Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + TypeConversions.fromLegacyInfoToDataType(indexFieldType))); + } + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(Row row) { + Object indexField = row.getField(indexFieldPos); + return indexPrefix.concat(indexField == null ? "null" : indexField.toString()).concat(indexSuffix); + } + }; + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field type + * ans parse index format from pattern. + */ + private static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + //time related types + supportedTypes.add(Types.LOCAL_DATE_TIME); + supportedTypes.add(Types.SQL_TIMESTAMP); + supportedTypes.add(Types.LOCAL_DATE); + supportedTypes.add(Types.SQL_DATE); + supportedTypes.add(Types.LOCAL_TIME); + supportedTypes.add(Types.SQL_TIME); + //general types + supportedTypes.add(Types.STRING); + supportedTypes.add(Types.SHORT); + supportedTypes.add(Types.INT); + supportedTypes.add(Types.LONG); + } + + static { + defaultFormats.put(Types.LOCAL_DATE_TIME, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(Types.SQL_TIMESTAMP, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(Types.LOCAL_DATE, "yyyy_MM_dd"); + defaultFormats.put(Types.SQL_DATE, "yyyy_MM_dd"); + defaultFormats.put(Types.LOCAL_TIME, "HH_mm_ss"); + defaultFormats.put(Types.SQL_TIME, "HH_mm_ss"); + } + + /** + * Validate the index field Type. + */ + void validateIndexFieldType(TypeInformation indexTypeInfo) { + if (!supportedTypes.contains(indexTypeInfo)) { + throw new IllegalArgumentException(String.format("Unsupported type %s of index field, " + + "Supported types are: %s", indexTypeInfo, supportedTypes)); + } + } + + /** + * Get the default date format. + */ + String getDefaultFormat(TypeInformation indexTypeInfo) { + return defaultFormats.get(indexTypeInfo); + } + + /** + * Check general dynamic index is enabled or not by index pattern. + */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException(String.format("Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", index)); + } + return count == 1; + } + + /** + * Check time extract dynamic index is enabled or not by index pattern. + */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** + * Extract dynamic index pattern string from index pattern string. + */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** + * Extract index field position in a fieldNames, return the field position. + */ + int extractIndexFieldPos(String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException(String.format("Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** + * Extract dateTime format by the date format that extracted from index pattern string. + */ + private String extractDateFormat(String index, TypeInformation indexTypeInfo) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(indexTypeInfo); + } + return format; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java new file mode 100644 index 00000000..054ee941 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java @@ -0,0 +1,35 @@ +/* + * 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.connectors.elasticsearch.index; + +import org.apache.flink.types.Row; + +/** + * A static {@link IndexGenerator} which generate fixed index name. + */ +public class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(Row row) { + return index; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index d82b2df1..ced65a00 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -22,6 +22,8 @@ import org.apache.flink.formats.json.JsonRowSerializationSchema; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.descriptors.Elasticsearch; @@ -80,7 +82,8 @@ public void testTableSink() { JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), - createTestSinkOptions()); + createTestSinkOptions(), + IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); // construct table sink using descriptors and table sink factory final Map elasticSearchProperties = createElasticSearchProperties(); @@ -106,7 +109,8 @@ public void testTableSinkWithLegacyProperties() { JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), - createTestSinkOptions()); + createTestSinkOptions(), + IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); // construct table sink using descriptors and table sink factory final Map elasticSearchProperties = createElasticSearchProperties(); @@ -197,7 +201,8 @@ protected abstract ElasticsearchUpsertTableSinkBase getExpectedTableSink( SerializationSchema serializationSchema, XContentType contentType, ActionRequestFailureHandler failureHandler, - Map sinkOptions); + Map sinkOptions, + IndexGenerator indexGenerator); // -------------------------------------------------------------------------------------------- // Helper classes diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java new file mode 100644 index 00000000..89a67bbd --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java @@ -0,0 +1,246 @@ +/* + * 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.connectors.elasticsearch.index; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.Row; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.List; + +/** + * Suite tests for {@link IndexGenerator}. + */ +public class IndexGeneratorTest { + + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + String[] fieldNames = new String[]{ + "id", + "item", + "log_ts", + "log_date", + "order_timestamp", + "log_time", + "local_datetime", + "local_date", + "local_time", + "note", + "status"}; + DataType[] dataTypes = new DataType[]{ + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.DATE().bridgedTo(java.sql.Date.class), + DataTypes.TIMESTAMP().bridgedTo(java.sql.Timestamp.class), + DataTypes.TIME().bridgedTo(java.sql.Time.class), + DataTypes.TIMESTAMP().bridgedTo(java.time.LocalDateTime.class), + DataTypes.DATE().bridgedTo(java.time.LocalDate.class), + DataTypes.TIME().bridgedTo(java.time.LocalTime.class), + DataTypes.STRING(), + DataTypes.BOOLEAN() + }; + schema = new TableSchema.Builder().fields(fieldNames, dataTypes).build(); + + rows = new ArrayList<>(); + rows.add(Row.of( + 1, + "apple", + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + Date.valueOf("2020-03-18"), + Timestamp.valueOf("2020-03-18 12:12:14"), + Time.valueOf("12:12:14"), + LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000), + LocalDate.of(2020, 3, 18), + LocalTime.of(12, 13, 14, 2000), + "test1", + true)); + rows.add(Row.of( + 2, + "peanut", + Timestamp.valueOf("2020-03-19 12:22:14").getTime(), + Date.valueOf("2020-03-19"), + Timestamp.valueOf("2020-03-19 12:22:21"), + Time.valueOf("12:22:21"), + LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000), + LocalDate.of(2020, 3, 19), + LocalTime.of(12, 13, 14, 2000), + "test2", + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDateTime() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDate() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalTime() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = "Unsupported type Boolean of index field, Supported types are:" + + " [LocalDateTime, Timestamp, LocalDate, Date, LocalTime, Time, String, Short, Integer, Long]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 021bbdd0..a5b5ef18 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -37,6 +37,8 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; import org.apache.flink.streaming.connectors.elasticsearch6.Elasticsearch6UpsertTableSink.DefaultRestClientFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.types.Row; @@ -61,6 +63,7 @@ public class Elasticsearch6UpsertTableSinkFactoryTest extends ElasticsearchUpser @Test public void testBuilder() { final TableSchema schema = createTestSchema(); + final IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator(INDEX, schema); final TestElasticsearch6UpsertTableSink testSink = new TestElasticsearch6UpsertTableSink( false, @@ -84,7 +87,7 @@ public void testBuilder() { final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), new ElasticsearchUpsertSinkFunction( - INDEX, + indexGenerator, DOC_TYPE, KEY_DELIMITER, KEY_NULL_LITERAL, @@ -101,7 +104,6 @@ public void testBuilder() { expectedBuilder.setBulkFlushMaxActions(1000); expectedBuilder.setBulkFlushMaxSizeMb(1); expectedBuilder.setRestClientFactory(new DefaultRestClientFactory(100, "/myapp")); - assertEquals(expectedBuilder, testSink.builder); } @@ -122,7 +124,8 @@ protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( SerializationSchema serializationSchema, XContentType contentType, ActionRequestFailureHandler failureHandler, - Map sinkOptions) { + Map sinkOptions, + IndexGenerator indexGenerator) { return new Elasticsearch6UpsertTableSink( isAppendOnly, schema, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index f3493b34..c24e3279 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -37,6 +37,8 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; +import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.types.Row; @@ -60,15 +62,16 @@ public class Elasticsearch7UpsertTableSinkFactoryTest extends ElasticsearchUpser @Test public void testBuilder() { final TableSchema schema = createTestSchema(); + final IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator(INDEX, schema); final TestElasticsearch7UpsertTableSink testSink = new TestElasticsearch7UpsertTableSink( false, schema, - Collections.singletonList(new Host(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), - ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, - ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, - ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, - ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, new DummyFailureHandler(), @@ -81,12 +84,12 @@ public void testBuilder() { testSink.consumeDataStream(dataStreamMock); final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( - Collections.singletonList(new HttpHost(ElasticsearchUpsertTableSinkFactoryTestBase.HOSTNAME, ElasticsearchUpsertTableSinkFactoryTestBase.PORT, ElasticsearchUpsertTableSinkFactoryTestBase.SCHEMA)), + Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), new ElasticsearchUpsertSinkFunction( - ElasticsearchUpsertTableSinkFactoryTestBase.INDEX, - ElasticsearchUpsertTableSinkFactoryTestBase.DOC_TYPE, - ElasticsearchUpsertTableSinkFactoryTestBase.KEY_DELIMITER, - ElasticsearchUpsertTableSinkFactoryTestBase.KEY_NULL_LITERAL, + indexGenerator, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), XContentType.JSON, Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, @@ -100,7 +103,6 @@ public void testBuilder() { expectedBuilder.setBulkFlushMaxActions(1000); expectedBuilder.setBulkFlushMaxSizeMb(1); expectedBuilder.setRestClientFactory(new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); - assertEquals(expectedBuilder, testSink.builder); } @@ -121,7 +123,8 @@ protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( SerializationSchema serializationSchema, XContentType contentType, ActionRequestFailureHandler failureHandler, - Map sinkOptions) { + Map sinkOptions, + IndexGenerator indexGenerator) { return new Elasticsearch7UpsertTableSink( isAppendOnly, schema, From a3053a1060eb98615e29f61b28f325efba05573d Mon Sep 17 00:00:00 2001 From: Yu Li Date: Fri, 1 May 2020 21:25:18 +0800 Subject: [PATCH 086/207] [FLINK-17483][legal] Update flink-sql-connector-elasticsearch7 NOTICE file to correctly reflect bundled dependencies --- .../src/main/resources/META-INF/NOTICE | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 9f68ec03..ea7772dc 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -6,10 +6,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) +- com.carrotsearch:hppc:0.8.1 - com.fasterxml.jackson.core:jackson-core:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 +- com.github.spullara.mustache.java:compiler:0.9.6 - commons-codec:commons-codec:1.10 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.4 @@ -34,11 +36,13 @@ This project bundles the following dependencies under the Apache Software Licens - org.elasticsearch:elasticsearch:7.5.1 - org.elasticsearch:elasticsearch-cli:7.5.1 - org.elasticsearch:elasticsearch-core:7.5.1 +- org.elasticsearch:elasticsearch-geo:7.5.1 - org.elasticsearch:elasticsearch-secure-sm:7.5.1 - org.elasticsearch:elasticsearch-x-content:7.5.1 - org.elasticsearch.client:elasticsearch-rest-client:7.5.1 - org.elasticsearch.client:elasticsearch-rest-high-level-client:7.5.1 - org.elasticsearch.plugin:aggs-matrix-stats-client:7.5.1 +- org.elasticsearch.plugin:lang-mustache-client:7.5.1 - org.elasticsearch.plugin:mapper-extras-client:7.5.1 - org.elasticsearch.plugin:parent-join-client:7.5.1 - org.elasticsearch.plugin:rank-eval-client:7.5.1 From 013286ef6ec907921071d8328bdc3359ca8faa55 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 15 May 2020 20:06:40 +0200 Subject: [PATCH 087/207] [FLINK-17027] Introduce a new Elasticsearch 7 connector with new property keys --- .../table/AbstractTimeIndexGenerator.java | 43 +++ .../table/ElasticsearchConfiguration.java | 152 ++++++++++ .../table/ElasticsearchOptions.java | 134 +++++++++ .../table/ElasticsearchValidationUtils.java | 93 ++++++ .../elasticsearch/table/IndexGenerator.java | 42 +++ .../table/IndexGeneratorBase.java | 54 ++++ .../table/IndexGeneratorFactory.java | 276 ++++++++++++++++++ .../elasticsearch/table/KeyExtractor.java | 131 +++++++++ .../elasticsearch/table/RequestFactory.java | 67 +++++ .../table/RowElasticsearchSinkFunction.java | 142 +++++++++ .../table/StaticIndexGenerator.java | 37 +++ .../ElasticsearchSinkTestBase.java | 42 +-- .../table/IndexGeneratorFactoryTest.java | 213 ++++++++++++++ .../elasticsearch/table/KeyExtractorTest.java | 130 +++++++++ .../elasticsearch/table/TestContext.java | 79 +++++ .../testutils/ElasticsearchResource.java | 78 +++++ .../flink-connector-elasticsearch7/pom.xml | 22 ++ .../table/Elasticsearch7Configuration.java | 71 +++++ .../table/Elasticsearch7DynamicSink.java | 252 ++++++++++++++++ .../Elasticsearch7DynamicSinkFactory.java | 154 ++++++++++ .../org.apache.flink.table.factories.Factory | 16 + .../Elasticsearch7DynamicSinkFactoryTest.java | 200 +++++++++++++ .../Elasticsearch7DynamicSinkITCase.java | 254 ++++++++++++++++ .../table/Elasticsearch7DynamicSinkTest.java | 195 +++++++++++++ 24 files changed, 2838 insertions(+), 39 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000..864d7fab --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java @@ -0,0 +1,43 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.time.format.DateTimeFormatter; + +/** + * Abstract class for time related {@link IndexGenerator}. + */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java new file mode 100644 index 00000000..48b848cd --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -0,0 +1,152 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.InstantiationUtil; + +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; + +/** + * Accessor methods to elasticsearch options. + */ +@Internal +class ElasticsearchConfiguration { + protected final ReadableConfig config; + private final ClassLoader classLoader; + + ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { + this.config = config; + this.classLoader = classLoader; + } + + public ActionRequestFailureHandler getFailureHandler() { + final ActionRequestFailureHandler failureHandler; + String value = config.get(FAILURE_HANDLER_OPTION); + switch (value.toUpperCase()) { + case "FAIL": + failureHandler = new NoOpFailureHandler(); + break; + case "IGNORE": + failureHandler = new IgnoringFailureHandler(); + break; + case "RETRY-REJECTED": + failureHandler = new RetryRejectedExecutionFailureHandler(); + break; + default: + try { + Class failureHandlerClass = Class.forName(value, false, classLoader); + failureHandler = (ActionRequestFailureHandler) InstantiationUtil.instantiate(failureHandlerClass); + } catch (ClassNotFoundException e) { + throw new ValidationException("Could not instantiate the failure handler class: " + value, e); + } + break; + } + return failureHandler; + } + + public String getDocumentType() { + return config.get(ElasticsearchOptions.DOCUMENT_TYPE_OPTION); + } + + public Optional getBulkFlushMaxActions() { + return config.getOptional(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + } + + public Optional getBulkFlushMaxSize() { + return config.getOptional(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).map(MemorySize::getMebiBytes); + } + + public Optional getBulkFlushInterval() { + return config.getOptional(BULK_FLUSH_INTERVAL_OPTION).map(Duration::toMillis); + } + + public boolean isBulkFlushBackoffEnabled() { + return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) != ElasticsearchOptions.BackOffType.DISABLED; + } + + public Optional getBulkFlushBackoffType() { + switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { + case CONSTANT: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); + case EXPONENTIAL: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + default: + return Optional.empty(); + } + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { + return !config.get(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION); + } + + public String getIndex() { + return config.get(ElasticsearchOptions.INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(ElasticsearchOptions.KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(ElasticsearchOptions.CONNECTION_PATH_PREFIX); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; + return Objects.equals(config, that.config) && + Objects.equals(classLoader, that.classLoader); + } + + @Override + public int hashCode() { + return Objects.hash(config, classLoader); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java new file mode 100644 index 00000000..c68ca688 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -0,0 +1,134 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; + +/** + * Options for {@link org.apache.flink.table.factories.DynamicTableSinkFactory} for Elasticsearch. + */ +public class ElasticsearchOptions { + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with + * {@code DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticseatch hosts to connect to."); + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription("Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + public static final ConfigOption FAILURE_HANDLER_OPTION = + ConfigOptions.key("failure-handler") + .stringType() + .defaultValue("fail") + .withDescription(Description.builder() + .text("Failure handling strategy in case a request to Elasticsearch fails") + .list( + text("\"fail\" (throws an exception if a request fails and thus causes a job failure),"), + text("\"ignore\" (ignores failures and drops the request),"), + text("\"retry_rejected\" (re-adds requests that have failed due to queue capacity saturation),"), + text("\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) + .build()); + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = + ConfigOptions.key("sink.flush-on-checkpoint") + .booleanType() + .defaultValue(true) + .withDescription("Disables flushing on checkpoint"); + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .noDefaultValue() + .withDescription("Maximum number of actions to buffer for each bulk request."); + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .noDefaultValue() + .withDescription("Maximum size of buffered actions per bulk request"); + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .noDefaultValue() + .withDescription("Bulk flush interval"); + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.back-off.strategy") + .enumType(BackOffType.class) + .defaultValue(BackOffType.DISABLED) + .withDescription("Backoff strategy"); + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.back-off.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.back-off.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = + ConfigOptions.key("connection.max-retry-timeout") + .durationType() + .noDefaultValue() + .withDescription("Maximum timeout between retries."); + public static final ConfigOption CONNECTION_PATH_PREFIX = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription("Elasticsearch connector requires to specify a format.\n" + + "The format must produce a valid json document. \n" + + "By default uses built-in 'json' format. Please refer to Table Formats section for more details."); + + private ElasticsearchOptions() { + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java new file mode 100644 index 00000000..b5caa765 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -0,0 +1,93 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; + +/** + * Utility methods for validating Elasticsearch properties. + */ +@Internal +class ElasticsearchValidationUtils { + + private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + } + + /** + * Checks that the table does not have primary key defined on illegal types. + * In Elasticsearch the primary key is used to calculate the Elasticsearch document id, + * which is a string of up to 512 bytes. It cannot have whitespaces. As of now it is calculated + * by concatenating the fields. Certain types do not have a good string representation to be used + * in this scenario. The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and + * {@link LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(TableSchema schema) { + schema.getPrimaryKey().ifPresent( + key -> { + List illegalTypes = key.getColumns() + .stream() + .map(fieldName -> { + LogicalType logicalType = schema.getFieldDataType(fieldName).get().getLogicalType(); + if (hasRoot(logicalType, LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType).getSourceType().getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) + .collect(Collectors.toList()); + + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.\n" + + " Elasticsearch sink does not support primary keys on columns of types: %s.", + illegalTypes, + ILLEGAL_PRIMARY_KEY_TYPES)); + } + } + ); + } + + private ElasticsearchValidationUtils() { + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java new file mode 100644 index 00000000..f5faf84c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -0,0 +1,42 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** + * This interface is responsible to generate index name from given {@link Row} record. + */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link #generate(RowData)} is called. + */ + default void open() {} + + /** + * Generate index name according the the given row. + */ + String generate(RowData row); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java new file mode 100644 index 00000000..5df3efdf --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java @@ -0,0 +1,54 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** + * Base class for {@link IndexGenerator}. + */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java new file mode 100644 index 00000000..e60be723 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -0,0 +1,276 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. 'myusers', + * all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in the + * record to dynamically generate a target index. You can also use '{field_name|date_format_string}' to + * convert a field value of TIMESTAMP/DATE/TIME type into the format specified by date_format_string. The + * date_format_string is compatible with {@link java.text.SimpleDateFormat}. For example, if the option + * value is 'myusers_{log_ts|yyyy-MM-dd}', then a record with log_ts field value 2020-03-27 12:25:55 will + * be written into "myusers-2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator(index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = createFormatFunction( + indexFieldType, + indexFieldLogicalTypeRoot); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix.concat(indexField == null ? "null" : indexField.toString()).concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, + LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L) + .format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException("TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException(String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field type + * ans parse index format from pattern. + */ + private static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + //time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + //general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** + * Validate the index field Type. + */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException(String.format("Unsupported type %s of index field, " + + "Supported types are: %s", logicalType, supportedTypes)); + } + } + + /** + * Get the default date format. + */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** + * Check general dynamic index is enabled or not by index pattern. + */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException(String.format("Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", index)); + } + return count == 1; + } + + /** + * Check time extract dynamic index is enabled or not by index pattern. + */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** + * Extract dynamic index pattern string from index pattern string. + */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** + * Extract index field position in a fieldNames, return the field position. + */ + int extractIndexFieldPos(String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException(String.format("Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** + * Extract dateTime format by the date format that extracted from index pattern string. + */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java new file mode 100644 index 00000000..db28ff19 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -0,0 +1,131 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** + * An extractor for a Elasticsearch key from a {@link RowData}. + */ +@Internal +class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor( + FieldFormatter[] fieldFormatters, + String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + private static class ColumnWithIndex { + public TableColumn column; + public int index; + + public ColumnWithIndex(TableColumn column, int index) { + this.column = column; + this.index = index; + } + + public LogicalType getType() { + return column.getType().getLogicalType(); + } + + public int getIndex() { + return index; + } + } + + public static Function createKeyExtractor( + TableSchema schema, + String keyDelimiter) { + return schema.getPrimaryKey().map(key -> { + Map namesToColumns = new HashMap<>(); + List tableColumns = schema.getTableColumns(); + for (int i = 0; i < schema.getFieldCount(); i++) { + TableColumn column = tableColumns.get(i); + namesToColumns.put(column.getName(), new ColumnWithIndex(column, i)); + } + + FieldFormatter[] fieldFormatters = key.getColumns() + .stream() + .map(namesToColumns::get) + .map(column -> toFormatter(column.index, column.getType())) + .toArray(FieldFormatter[]::new); + + return (Function) new KeyExtractor( + fieldFormatters, + keyDelimiter + ); + }).orElseGet(() -> (Function & Serializable) (row) -> null); + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter( + type, + index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java new file mode 100644 index 00000000..35d69eb6 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.Serializable; + +/** + * For version-agnostic creating of {@link ActionRequest}s. + */ +@Internal +interface RequestFactory extends Serializable { + /** + * Creates an update request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. + * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest( + String index, + String docType, + String key); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java new file mode 100644 index 00000000..4eaba480 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -0,0 +1,142 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.function.Function; + +/** + * Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. + */ +@Internal +class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1L; + + private final IndexGenerator indexGenerator; + private final String docType; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final Function createKey; + + public RowElasticsearchSinkFunction( + IndexGenerator indexGenerator, + @Nullable String docType, // this is deprecated in es 7+ + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + Function createKey) { + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = docType; + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.createKey = Preconditions.checkNotNull(createKey); + } + + @Override + public void process( + RowData element, + RuntimeContext ctx, + RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = requestFactory.createUpdateRequest( + indexGenerator.generate(row), + docType, + key, + contentType, + document); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = requestFactory.createIndexRequest( + indexGenerator.generate(row), + docType, + key, + contentType, + document); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = requestFactory.createDeleteRequest( + indexGenerator.generate(row), + docType, + key); + indexer.add(deleteRequest); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) && + Objects.equals(docType, that.docType) && + Objects.equals(serializationSchema, that.serializationSchema) && + contentType == that.contentType && + Objects.equals(requestFactory, that.requestFactory) && + Objects.equals(createKey, that.createKey); + } + + @Override + public int hashCode() { + return Objects.hash(indexGenerator, docType, serializationSchema, contentType, requestFactory, createKey); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java new file mode 100644 index 00000000..196b64ce --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java @@ -0,0 +1,37 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** + * A static {@link IndexGenerator} which generate fixed index name. + */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 8b568962..35bdca7b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -21,17 +21,12 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.client.Client; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.Collections; import java.util.HashMap; @@ -49,41 +44,10 @@ */ public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { - private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkTestBase.class); - protected static final String CLUSTER_NAME = "test-cluster"; - protected static EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; - @ClassRule - public static TemporaryFolder tempFolder = new TemporaryFolder(); - - @BeforeClass - public static void prepare() throws Exception { - - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Starting embedded Elasticsearch node "); - LOG.info("-------------------------------------------------------------------------"); - - // dynamically load version-specific implementation of the Elasticsearch embedded node environment - Class clazz = Class.forName( - "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); - embeddedNodeEnv = (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); - - embeddedNodeEnv.start(tempFolder.newFolder(), CLUSTER_NAME); - - } - - @AfterClass - public static void shutdown() throws Exception { - - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Shutting down embedded Elasticsearch node "); - LOG.info("-------------------------------------------------------------------------"); - - embeddedNodeEnv.close(); - - } + public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource(CLUSTER_NAME); /** * Tests that the Elasticsearch sink works properly with json. @@ -128,7 +92,7 @@ private void runElasticSearchSinkTest(Function rows; + + @Before + public void prepareData() { + schema = new TableSchema.Builder() + .field("id", DataTypes.INT()) + .field("item", DataTypes.STRING()) + .field("log_ts", DataTypes.BIGINT()) + .field("log_date", DataTypes.DATE()) + .field("log_time", DataTypes.TIME()) + .field("order_timestamp", DataTypes.TIMESTAMP()) + .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("status", DataTypes.BOOLEAN()) + .build(); + + rows = new ArrayList<>(); + rows.add(GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-18").toEpochDay(), + (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), + TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), + true)); + rows.add(GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-19").toEpochDay(), + (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), + TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = IndexGeneratorFactory + .createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "my-index-{log_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "my-index-{local_timestamp|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = IndexGeneratorFactory + .createIndexGenerator( + "index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( + "my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java new file mode 100644 index 00000000..96b6e3ec --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java @@ -0,0 +1,130 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; + +import org.junit.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; + +/** + * Tests for {@link KeyExtractor}. + */ +public class KeyExtractorTest { + @Test + public void testSimpleKey() { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .primaryKey("a") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, equalTo("12")); + } + + @Test + public void testNoPrimaryKey() { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, nullValue()); + } + + @Test + public void testTwoFieldsKey() { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "c") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12")) + )); + assertThat(key, equalTo("12_2012-12-12T12:12:12")); + } + + @Test + public void testAllTypesKey() { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.TINYINT().notNull()) + .field("b", DataTypes.SMALLINT().notNull()) + .field("c", DataTypes.INT().notNull()) + .field("d", DataTypes.BIGINT().notNull()) + .field("e", DataTypes.BOOLEAN().notNull()) + .field("f", DataTypes.FLOAT().notNull()) + .field("g", DataTypes.DOUBLE().notNull()) + .field("h", DataTypes.STRING().notNull()) + .field("i", DataTypes.TIMESTAMP().notNull()) + .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) + .field("k", DataTypes.TIME().notNull()) + .field("l", DataTypes.DATE().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay() + )); + assertThat( + key, + equalTo("1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java new file mode 100644 index 00000000..e4978fc1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -0,0 +1,79 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.factories.DynamicTableFactory; + +import java.util.HashMap; +import java.util.Map; + +/** + * A utility class for mocking {@link DynamicTableFactory.Context}. + */ +class TestContext { + private TableSchema schema; + private Map properties = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(TableSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new DynamicTableFactory.Context() { + @Override + public ObjectIdentifier getObjectIdentifier() { + return null; + } + + @Override + public CatalogTable getCatalogTable() { + return new CatalogTableImpl( + schema, + properties, + "" + ); + } + + @Override + public ReadableConfig getConfiguration() { + return null; + } + + @Override + public ClassLoader getClassLoader() { + return TestContext.class.getClassLoader(); + } + }; + } + + public TestContext withOption(String key, String value) { + properties.put(key, value); + return this; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java new file mode 100644 index 00000000..6f185d30 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.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 org.apache.flink.streaming.connectors.elasticsearch.testutils; + +import org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironment; +import org.apache.flink.util.InstantiationUtil; + +import org.elasticsearch.client.Client; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A resource that starts an embedded elasticsearch cluster. + */ +public class ElasticsearchResource extends ExternalResource { + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchResource.class); + private EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; + private final TemporaryFolder tempFolder = new TemporaryFolder(); + + private final String clusterName; + + public ElasticsearchResource(String clusterName) { + this.clusterName = clusterName; + } + + @Override + protected void before() throws Throwable { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + // dynamically load version-specific implementation of the Elasticsearch embedded node environment + Class clazz = Class.forName( + "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); + embeddedNodeEnv = (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); + + tempFolder.create(); + embeddedNodeEnv.start(tempFolder.newFolder(), clusterName); + } + + @Override + protected void after() { + + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shutting down embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); + + try { + embeddedNodeEnv.close(); + tempFolder.delete(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public Client getClient() { + return embeddedNodeEnv.getClient(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 5f25f46c..ef340b9b 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -139,6 +139,14 @@ under the License. test + + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${project.version} + test + + org.apache.flink @@ -148,4 +156,18 @@ under the License. + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + + + + diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java new file mode 100644 index 00000000..0aff5756 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -0,0 +1,71 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; + +/** + * Elasticsearch 7 specific configuration. + */ +@Internal +final class Elasticsearch7Configuration extends ElasticsearchConfiguration { + Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch7Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, + HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, + HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, + HOSTS_OPTION.key()), e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java new file mode 100644 index 00000000..4076b63d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -0,0 +1,252 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a logical + * description. + */ +@Internal +final class Elasticsearch7DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7DynamicSink.Elasticsearch7RequestFactory(); + + private final SinkFormat> format; + private final TableSchema schema; + private final Elasticsearch7Configuration config; + + public Elasticsearch7DynamicSink( + SinkFormat> format, + Elasticsearch7Configuration config, + TableSchema schema) { + this(format, config, schema, (ElasticsearchSink.Builder::new)); + } + + //-------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + //-------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, + RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch7DynamicSink( + SinkFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.builderProvider = builderProvider; + } + + //-------------------------------------------------------------- + // End of hack to make configuration testing possible + //-------------------------------------------------------------- + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = this.format.createSinkFormat(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), + null, // this is deprecated in es 7+ + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()) + ); + + final ElasticsearchSink.Builder builder = builderProvider.createBuilder( + config.getHosts(), + upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + config.getBulkFlushMaxActions().ifPresent(builder::setBulkFlushMaxActions); + config.getBulkFlushMaxSize().ifPresent(builder::setBulkFlushMaxSizeMb); + config.getBulkFlushInterval().ifPresent(builder::setBulkFlushInterval); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + config.getPathPrefix() + .ifPresent(pathPrefix -> builder.setRestClientFactory(new DefaultRestClientFactory(pathPrefix))); + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** + * Serializable {@link RestClientFactory} used by the sink. + */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index) + .id(key) + .source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + return Objects.equals(format, that.format) && + Objects.equals(schema, that.schema) && + Objects.equals(config, that.config) && + Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java new file mode 100644 index 00000000..055989b1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -0,0 +1,154 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; + +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; + +/** + * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. + */ +@Internal +public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = Stream.of( + HOSTS_OPTION, + INDEX_OPTION + ).collect(Collectors.toSet()); + private static final Set> optionalOptions = Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION + ).collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + + final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + final SinkFormat> format = helper.discoverSinkFormat( + SerializationFormatFactory.class, + FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable() + .getOptions() + .forEach(configuration::setString); + Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + validateOptions(config, configuration); + + return new Elasticsearch7DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema)); + } + + private void validateOptions(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validateOptions( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + validateOptions( + config.getBulkFlushMaxActions().map(maxActions -> maxActions >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1 character. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + config.getBulkFlushMaxActions().get()) + ); + validateOptions( + config.getBulkFlushMaxSize().map(maxSize -> maxSize >= 1024 * 1024).orElse(true), + () -> String.format( + "'%s' must be at least 1mb character. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) + ); + validateOptions( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get()) + ); + } + + private static void validateOptions(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-7"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..10e4846e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java new file mode 100644 index 00000000..a830fa30 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -0,0 +1,200 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** + * Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. + */ +public class Elasticsearch7DynamicSinkFactoryTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .build() + ); + } + + @Test + public void validateWrongIndex() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption("index", "") + .withOption("hosts", "http://localhost:12345") + .build() + ); + } + + @Test + public void validateWrongHosts() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption("index", "MyIndex") + .withOption("hosts", "wrong-host") + .build() + ); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be at least 1mb character. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + .build() + ); + } + + @Test + public void validateWrongRetries() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.back-off.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-actions' must be at least 1 character. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Invalid value for option 'sink.bulk-flush.back-off.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + .build() + ); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.ARRAY(DataTypes.BIGINT().notNull()).notNull()) + .field("c", DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING()).notNull()) + .field("d", DataTypes.MULTISET(DataTypes.BIGINT().notNull()).notNull()) + .field("e", DataTypes.ROW(DataTypes.FIELD("a", DataTypes.BIGINT())).notNull()) + .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field("g", DataTypes.BYTES().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g") + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + .build() + ); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java new file mode 100644 index 00000000..3b667dc1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -0,0 +1,254 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.search.SearchHits; +import org.junit.ClassRule; +import org.junit.Test; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * IT tests for {@link Elasticsearch7DynamicSink}. + */ +public class Elasticsearch7DynamicSinkITCase { + + @ClassRule + public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource("es-dynamic-sink-it-test"); + + @Test + public void testWritingDocuments() throws Exception { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.TIME()) + .field("c", DataTypes.STRING().notNull()) + .field("d", DataTypes.FLOAT()) + .field("e", DataTypes.TINYINT().notNull()) + .field("f", DataTypes.DATE()) + .field("g", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "g") + .build(); + GenericRowData rowData = GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = (SinkFunctionProvider) sinkFactory.createDynamicTableSink( + context() + .withSchema(schema) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + .build() + ).getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "table-api"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL," + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")) + ).executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "no-primary-key"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")) + ).executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); + SearchHits hits; + do { + hits = client.prepareSearch(index) + .execute() + .actionGet() + .getHits(); + if (hits.getTotalHits().value == 0) { + Thread.sleep(100); + } + } while (hits.getTotalHits().value == 0 && deadline.hasTimeLeft()); + + Map result = hits.getAt(0).getSourceAsMap(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(result, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java new file mode 100644 index 00000000..466ede37 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -0,0 +1,195 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link Elasticsearch7DynamicSink} parameters. + */ +public class Elasticsearch7DynamicSinkTest { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( + new DummySinkFormat(), + new Elasticsearch7Configuration(getConfig(), this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy).setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString(ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); + configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, + RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = Mockito.spy((ElasticsearchSink) invocation.callRealMethod()); + return sinkSpy; + } + ).when(builderSpy).build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummySinkFormat implements SinkFormat> { + @Override + public SerializationSchema createSinkFormat( + DynamicTableSink.Context context, + DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { + return null; + } + } + + /** + * Custom failure handler for testing. + */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} From 51139e2b1d0739900ff4759c08f5e6e18bbf5136 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Fri, 15 May 2020 20:06:46 +0200 Subject: [PATCH 088/207] [FLINK-17027] Introduce a new Elasticsearch 6 connector with new property keys This closes #12184 --- .../flink-connector-elasticsearch6/pom.xml | 22 ++ .../table/Elasticsearch6Configuration.java | 80 ++++++ .../table/Elasticsearch6DynamicSink.java | 251 +++++++++++++++++ .../Elasticsearch6DynamicSinkFactory.java | 155 +++++++++++ .../org.apache.flink.table.factories.Factory | 16 ++ .../Elasticsearch6DynamicSinkFactoryTest.java | 207 ++++++++++++++ .../Elasticsearch6DynamicSinkITCase.java | 262 ++++++++++++++++++ .../table/Elasticsearch6DynamicSinkTest.java | 195 +++++++++++++ 8 files changed, 1188 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 25a9f5a7..272f9af0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -145,6 +145,14 @@ under the License. test + + + org.apache.flink + flink-table-planner-blink_${scala.binary.version} + ${project.version} + test + + org.apache.flink @@ -154,4 +162,18 @@ under the License. + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + 1 + + + + diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java new file mode 100644 index 00000000..c06898e8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -0,0 +1,80 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; + +/** + * Elasticsearch 7 specific configuration. + */ +@Internal +final class Elasticsearch6Configuration extends ElasticsearchConfiguration { + Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch6Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+	 *     connector.hosts = http://host_name:9092;http://host_name:9093
+	 * 
+ */ + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, + HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, + HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException(String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, + HOSTS_OPTION.key()), e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java new file mode 100644 index 00000000..eadf659f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -0,0 +1,251 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a logical + * description. + */ +@PublicEvolving +final class Elasticsearch6DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + + private final SinkFormat> format; + private final TableSchema schema; + private final Elasticsearch6Configuration config; + + public Elasticsearch6DynamicSink( + SinkFormat> format, + Elasticsearch6Configuration config, + TableSchema schema) { + this(format, config, schema, (ElasticsearchSink.Builder::new)); + } + + //-------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + //-------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, + RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch6DynamicSink( + SinkFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.builderProvider = builderProvider; + } + + //-------------------------------------------------------------- + // End of hack to make configuration testing possible + //-------------------------------------------------------------- + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = this.format.createSinkFormat(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), + config.getDocumentType(), + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()) + ); + + final ElasticsearchSink.Builder builder = builderProvider.createBuilder( + config.getHosts(), + upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + config.getBulkFlushMaxActions().ifPresent(builder::setBulkFlushMaxActions); + config.getBulkFlushMaxSize().ifPresent(builder::setBulkFlushMaxSizeMb); + config.getBulkFlushInterval().ifPresent(builder::setBulkFlushInterval); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + config.getPathPrefix() + .ifPresent(pathPrefix -> builder.setRestClientFactory(new DefaultRestClientFactory(pathPrefix))); + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** + * Serializable {@link RestClientFactory} used by the sink. + */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index, docType, index) + .source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; + return Objects.equals(format, that.format) && + Objects.equals(schema, that.schema) && + Objects.equals(config, that.config) && + Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java new file mode 100644 index 00000000..65c90b5e --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -0,0 +1,155 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; + +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; + +/** + * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. + */ +@Internal +public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = Stream.of( + HOSTS_OPTION, + INDEX_OPTION, + DOCUMENT_TYPE_OPTION + ).collect(Collectors.toSet()); + private static final Set> optionalOptions = Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION + ).collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + final SinkFormat> format = helper.discoverSinkFormat( + SerializationFormatFactory.class, + FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable() + .getOptions() + .forEach(configuration::setString); + Elasticsearch6Configuration config = new Elasticsearch6Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch6DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema)); + } + + private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + validate( + config.getBulkFlushMaxActions().map(maxActions -> maxActions >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1 character. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + config.getBulkFlushMaxActions().get()) + ); + validate( + config.getBulkFlushMaxSize().map(maxSize -> maxSize >= 1024 * 1024).orElse(true), + () -> String.format( + "'%s' must be at least 1mb character. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) + ); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get()) + ); + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-6"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000..29a85938 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# 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. + +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java new file mode 100644 index 00000000..f1be1b21 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -0,0 +1,207 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** + * Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. + */ +public class Elasticsearch6DynamicSinkFactoryTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .build() + ); + } + + @Test + public void validateWrongIndex() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption("index", "") + .withOption("document-type", "MyType") + .withOption("hosts", "http://localhost:12345") + .build() + ); + } + + @Test + public void validateWrongHosts() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption("index", "MyIndex") + .withOption("document-type", "MyType") + .withOption("hosts", "wrong-host") + .build() + ); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be at least 1mb character. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + .build() + ); + } + + @Test + public void validateWrongRetries() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.back-off.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-actions' must be at least 1 character. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "0") + .build() + ); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Invalid value for option 'sink.bulk-flush.back-off.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + .build() + ); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.ARRAY(DataTypes.BIGINT().notNull()).notNull()) + .field("c", DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING()).notNull()) + .field("d", DataTypes.MULTISET(DataTypes.BIGINT().notNull()).notNull()) + .field("e", DataTypes.ROW(DataTypes.FIELD("a", DataTypes.BIGINT())).notNull()) + .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field("g", DataTypes.BYTES().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g") + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + .build() + ); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java new file mode 100644 index 00000000..3c096532 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -0,0 +1,262 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.search.SearchHits; +import org.junit.ClassRule; +import org.junit.Test; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** + * IT tests for {@link Elasticsearch6DynamicSink}. + */ +public class Elasticsearch6DynamicSinkITCase { + + @ClassRule + public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource("es-6-dynamic-sink-tests"); + + @Test + public void testWritingDocuments() throws Exception { + TableSchema schema = TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.TIME()) + .field("c", DataTypes.STRING().notNull()) + .field("d", DataTypes.FLOAT()) + .field("e", DataTypes.TINYINT().notNull()) + .field("f", DataTypes.DATE()) + .field("g", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "g") + .build(); + GenericRowData rowData = GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + String myType = "MyType"; + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = (SinkFunctionProvider) sinkFactory.createDynamicTableSink( + context() + .withSchema(schema) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + .build() + ).getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "table-api"; + String myType = "MyType"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")) + ).executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "no-primary-key"; + String myType = "MyType"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")) + ).executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); + SearchHits hits; + do { + hits = client.prepareSearch(index) + .execute() + .actionGet() + .getHits(); + if (hits.getTotalHits() == 0) { + Thread.sleep(100); + } + } while (hits.getTotalHits() == 0 && deadline.hasTimeLeft()); + + Map result = hits.getAt(0).getSourceAsMap(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12Z"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12T12:12:12Z"); + assertThat(result, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java new file mode 100644 index 00000000..df541471 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -0,0 +1,195 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.verify; + +/** + * Tests for {@link Elasticsearch6DynamicSink} parameters. + */ +public class Elasticsearch6DynamicSinkTest { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( + new DummySinkFormat(), + new Elasticsearch6Configuration(getConfig(), this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy).setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString(ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); + configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, + RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = Mockito.spy((ElasticsearchSink) invocation.callRealMethod()); + return sinkSpy; + } + ).when(builderSpy).build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummySinkFormat implements SinkFormat> { + @Override + public SerializationSchema createSinkFormat( + DynamicTableSink.Context context, + DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { + return null; + } + } + + /** + * Custom failure handler for testing. + */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} From 52afd8b0860c73e645b5e3e9e744c7547980c7a6 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Mon, 18 May 2020 13:36:46 +0200 Subject: [PATCH 089/207] Update version to 1.12-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index f5c4252b..8ada7132 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 272f9af0..52f9cf6c 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index ef340b9b..d0385e5f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index f0a7a643..97166ac6 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index d1e289dc..0b165461 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.11-SNAPSHOT + 1.12-SNAPSHOT .. From 215186d866148d8c8a42f9703c38f0db7c8d7c83 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 19 May 2020 09:30:25 +0200 Subject: [PATCH 090/207] [FLINK-17027][hotfix] Rename back-off infix to backoff in new Elasticsearch properties --- .../elasticsearch/table/ElasticsearchOptions.java | 6 +++--- .../table/Elasticsearch6DynamicSinkFactoryTest.java | 4 ++-- .../table/Elasticsearch7DynamicSinkFactoryTest.java | 4 ++-- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index c68ca688..176414d6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -97,17 +97,17 @@ public enum BackOffType { .noDefaultValue() .withDescription("Bulk flush interval"); public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = - ConfigOptions.key("sink.bulk-flush.back-off.strategy") + ConfigOptions.key("sink.bulk-flush.backoff.strategy") .enumType(BackOffType.class) .defaultValue(BackOffType.DISABLED) .withDescription("Backoff strategy"); public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = - ConfigOptions.key("sink.bulk-flush.back-off.max-retries") + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") .intType() .noDefaultValue() .withDescription("Maximum number of retries."); public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = - ConfigOptions.key("sink.bulk-flush.back-off.delay") + ConfigOptions.key("sink.bulk-flush.backoff.delay") .durationType() .noDefaultValue() .withDescription("Delay between each backoff attempt."); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index f1be1b21..6d0878f2 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -122,7 +122,7 @@ public void validateWrongRetries() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.back-off.max-retries' must be at least 1. Got: 0"); + "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -162,7 +162,7 @@ public void validateWrongBackoffDelay() { thrown.expect(ValidationException.class); thrown.expectMessage( - "Invalid value for option 'sink.bulk-flush.back-off.delay'."); + "Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index a830fa30..4fe32142 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -118,7 +118,7 @@ public void validateWrongRetries() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.back-off.max-retries' must be at least 1. Got: 0"); + "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -156,7 +156,7 @@ public void validateWrongBackoffDelay() { thrown.expect(ValidationException.class); thrown.expectMessage( - "Invalid value for option 'sink.bulk-flush.back-off.delay'."); + "Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() From 004d18392caf95fa97b5c51e3354b352db884f50 Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Thu, 28 May 2020 10:32:53 +0800 Subject: [PATCH 091/207] [FLINK-16021][table-common] Fix DescriptorProperties.putTableSchema does not include PRIMARY KEY This closes #12275 --- .../ElasticsearchUpsertTableSinkFactoryBase.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index efbbb7e6..a20cb330 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -53,7 +53,7 @@ import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.DescriptorProperties.TABLE_SCHEMA_EXPR; +import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; @@ -149,13 +149,17 @@ public List supportedProperties() { properties.add(SCHEMA + ".#." + SCHEMA_TYPE); properties.add(SCHEMA + ".#." + SCHEMA_NAME); // computed column - properties.add(SCHEMA + ".#." + TABLE_SCHEMA_EXPR); + properties.add(SCHEMA + ".#." + EXPR); // watermark properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); + // table constraint + properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); + properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); + // format wildcard properties.add(FORMAT + ".*"); From 78528343e2d81cb86e66c88031236ec09749a65d Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 1 Jun 2020 10:32:40 +0800 Subject: [PATCH 092/207] [FLINK-17887][table][connector] Improve interface of ScanFormatFactory and SinkFormatFactory We improved the interfaces with the following changes: 1. Have a common interface DynamicTableSource.Context, and make Context of ScanTableSource and LookupTableSource extend it, and rename them to LookupContext and ScanContext 2. Change parameter of ScanFormat.createScanFormat from ScanTableSource.Context to DynamicTableSource.Context 3. Rename ScanFormat.createScanFormat to DecodingFormat#createRuntimeDecoder() 4. Rename SinkFormat.createSinkFormat to EncodingFormat#createRuntimeEncoder() 5. Rename ScanFormatFactory to DecodingFormatFactory 6. Rename SinkFormatFactory to EncodingFormatFactory This closes #12320 --- .../elasticsearch/table/Elasticsearch6DynamicSink.java | 10 +++++----- .../table/Elasticsearch6DynamicSinkFactory.java | 4 ++-- .../table/Elasticsearch6DynamicSinkTest.java | 8 ++++---- .../elasticsearch/table/Elasticsearch7DynamicSink.java | 10 +++++----- .../table/Elasticsearch7DynamicSinkFactory.java | 4 ++-- .../table/Elasticsearch7DynamicSinkTest.java | 8 ++++---- 6 files changed, 22 insertions(+), 22 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index eadf659f..bedfbef8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.RowData; @@ -52,12 +52,12 @@ final class Elasticsearch6DynamicSink implements DynamicTableSink { @VisibleForTesting static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); - private final SinkFormat> format; + private final EncodingFormat> format; private final TableSchema schema; private final Elasticsearch6Configuration config; public Elasticsearch6DynamicSink( - SinkFormat> format, + EncodingFormat> format, Elasticsearch6Configuration config, TableSchema schema) { this(format, config, schema, (ElasticsearchSink.Builder::new)); @@ -83,7 +83,7 @@ ElasticsearchSink.Builder createBuilder( } Elasticsearch6DynamicSink( - SinkFormat> format, + EncodingFormat> format, Elasticsearch6Configuration config, TableSchema schema, ElasticSearchBuilderProvider builderProvider) { @@ -111,7 +111,7 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { @Override public SinkFunctionProvider getSinkRuntimeProvider(Context context) { return () -> { - SerializationSchema format = this.format.createSinkFormat(context, schema.toRowDataType()); + SerializationSchema format = this.format.createRuntimeEncoder(context, schema.toRowDataType()); final RowElasticsearchSinkFunction upsertFunction = new RowElasticsearchSinkFunction( diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 65c90b5e..c5d9c893 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DynamicTableSinkFactory; @@ -84,7 +84,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); - final SinkFormat> format = helper.discoverSinkFormat( + final EncodingFormat> format = helper.discoverEncodingFormat( SerializationFormatFactory.class, FORMAT_OPTION); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index df541471..1708efc4 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; @@ -65,7 +65,7 @@ public void testBuilder() { BuilderProvider provider = new BuilderProvider(); final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( - new DummySinkFormat(), + new DummyEncodingFormat(), new Elasticsearch6Configuration(getConfig(), this.getClass().getClassLoader()), schema, provider @@ -141,9 +141,9 @@ public byte[] serialize(RowData element) { } } - private static class DummySinkFormat implements SinkFormat> { + private static class DummyEncodingFormat implements EncodingFormat> { @Override - public SerializationSchema createSinkFormat( + public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { return DummySerializationSchema.INSTANCE; diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 4076b63d..408673ef 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -25,7 +25,7 @@ import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.RowData; @@ -52,12 +52,12 @@ final class Elasticsearch7DynamicSink implements DynamicTableSink { @VisibleForTesting static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7DynamicSink.Elasticsearch7RequestFactory(); - private final SinkFormat> format; + private final EncodingFormat> format; private final TableSchema schema; private final Elasticsearch7Configuration config; public Elasticsearch7DynamicSink( - SinkFormat> format, + EncodingFormat> format, Elasticsearch7Configuration config, TableSchema schema) { this(format, config, schema, (ElasticsearchSink.Builder::new)); @@ -83,7 +83,7 @@ ElasticsearchSink.Builder createBuilder( } Elasticsearch7DynamicSink( - SinkFormat> format, + EncodingFormat> format, Elasticsearch7Configuration config, TableSchema schema, ElasticSearchBuilderProvider builderProvider) { @@ -111,7 +111,7 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { @Override public SinkFunctionProvider getSinkRuntimeProvider(Context context) { return () -> { - SerializationSchema format = this.format.createSinkFormat(context, schema.toRowDataType()); + SerializationSchema format = this.format.createRuntimeEncoder(context, schema.toRowDataType()); final RowElasticsearchSinkFunction upsertFunction = new RowElasticsearchSinkFunction( diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 055989b1..ae7a9fd1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DynamicTableSinkFactory; @@ -83,7 +83,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); - final SinkFormat> format = helper.discoverSinkFormat( + final EncodingFormat> format = helper.discoverEncodingFormat( SerializationFormatFactory.class, FORMAT_OPTION); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index 466ede37..c972cee4 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.SinkFormat; +import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; @@ -65,7 +65,7 @@ public void testBuilder() { BuilderProvider provider = new BuilderProvider(); final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( - new DummySinkFormat(), + new DummyEncodingFormat(), new Elasticsearch7Configuration(getConfig(), this.getClass().getClassLoader()), schema, provider @@ -141,9 +141,9 @@ public byte[] serialize(RowData element) { } } - private static class DummySinkFormat implements SinkFormat> { + private static class DummyEncodingFormat implements EncodingFormat> { @Override - public SerializationSchema createSinkFormat( + public SerializationSchema createRuntimeEncoder( DynamicTableSink.Context context, DataType consumedDataType) { return DummySerializationSchema.INSTANCE; From ab320049f086a34c35fd4bb63801297113c6ea51 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 2 Jun 2020 08:51:35 +0200 Subject: [PATCH 093/207] [FLINK-18052] Increase timeout for ES Search API in IT Cases This closes #12434 --- .../table/Elasticsearch6DynamicSinkITCase.java | 12 ++++++++---- .../table/Elasticsearch7DynamicSinkITCase.java | 12 ++++++++---- 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 3c096532..41a67f62 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -219,17 +219,21 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { // search API does not return documents that were not indexed, we might need to query // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); SearchHits hits; do { hits = client.prepareSearch(index) .execute() .actionGet() .getHits(); - if (hits.getTotalHits() == 0) { - Thread.sleep(100); + if (hits.getTotalHits() < 1) { + Thread.sleep(200); } - } while (hits.getTotalHits() == 0 && deadline.hasTimeLeft()); + } while (hits.getTotalHits() < 1 && deadline.hasTimeLeft()); + + if (hits.getTotalHits() < 1) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } Map result = hits.getAt(0).getSourceAsMap(); Map expectedMap = new HashMap<>(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 3b667dc1..dbd1ff90 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -211,17 +211,21 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { // search API does not return documents that were not indexed, we might need to query // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(1)); + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); SearchHits hits; do { hits = client.prepareSearch(index) .execute() .actionGet() .getHits(); - if (hits.getTotalHits().value == 0) { - Thread.sleep(100); + if (hits.getTotalHits().value < 1) { + Thread.sleep(200); } - } while (hits.getTotalHits().value == 0 && deadline.hasTimeLeft()); + } while (hits.getTotalHits().value < 1 && deadline.hasTimeLeft()); + + if (hits.getTotalHits().value < 1) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } Map result = hits.getAt(0).getSourceAsMap(); Map expectedMap = new HashMap<>(); From 2bb07acc2cda275329e10843369689026f0b5d64 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 2 Jun 2020 17:09:26 +0200 Subject: [PATCH 094/207] [FLINK-18006] Always overwrite RestClientFactory in ElasticsearchXDynamicSink We always overwrite the RestClientFactory in order to workaround an issue with shading classes in lambdas deserialization method. That way we never use the default lambda from ElasticsearchSink$Builder which cannot be deserialized when used from a flink-sql-connector-elasticsearch module due to shading. This closes #12455 --- .../elasticsearch/table/Elasticsearch6DynamicSink.java | 5 +++-- .../elasticsearch/table/Elasticsearch7DynamicSink.java | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index bedfbef8..680cb2c3 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -136,8 +136,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); - config.getPathPrefix() - .ifPresent(pathPrefix -> builder.setRestClientFactory(new DefaultRestClientFactory(pathPrefix))); + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); final ElasticsearchSink sink = builder.build(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 408673ef..7aa52eaf 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -136,8 +136,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); - config.getPathPrefix() - .ifPresent(pathPrefix -> builder.setRestClientFactory(new DefaultRestClientFactory(pathPrefix))); + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); final ElasticsearchSink sink = builder.build(); From 9f502e5f466646c79f23d7bc192ceeeaf314f312 Mon Sep 17 00:00:00 2001 From: liuyongvs Date: Tue, 9 Jun 2020 16:40:16 +0800 Subject: [PATCH 095/207] [FLINK-18208] Fix flink ES connector typos This closes #12543 --- .../table/Elasticsearch6DynamicSink.java | 4 ++-- .../table/Elasticsearch7DynamicSinkFactory.java | 14 +++++++------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index 680cb2c3..01b5f47f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -50,7 +50,7 @@ @PublicEvolving final class Elasticsearch6DynamicSink implements DynamicTableSink { @VisibleForTesting - static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); private final EncodingFormat> format; private final TableSchema schema; @@ -200,7 +200,7 @@ public int hashCode() { /** * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. */ - private static class Elasticsearch7RequestFactory implements RequestFactory { + private static class Elasticsearch6RequestFactory implements RequestFactory { @Override public UpdateRequest createUpdateRequest( String index, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index ae7a9fd1..320c894b 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -94,7 +94,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { .forEach(configuration::setString); Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration, context.getClassLoader()); - validateOptions(config, configuration); + validate(config, configuration); return new Elasticsearch7DynamicSink( format, @@ -102,27 +102,27 @@ public DynamicTableSink createDynamicTableSink(Context context) { TableSchemaUtils.getPhysicalSchema(tableSchema)); } - private void validateOptions(Elasticsearch7Configuration config, Configuration originalConfiguration) { + private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { config.getFailureHandler(); // checks if we can instantiate the custom failure handler config.getHosts(); // validate hosts - validateOptions( + validate( config.getIndex().length() >= 1, () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); - validateOptions( + validate( config.getBulkFlushMaxActions().map(maxActions -> maxActions >= 1).orElse(true), () -> String.format( "'%s' must be at least 1 character. Got: %s", BULK_FLUSH_MAX_ACTIONS_OPTION.key(), config.getBulkFlushMaxActions().get()) ); - validateOptions( + validate( config.getBulkFlushMaxSize().map(maxSize -> maxSize >= 1024 * 1024).orElse(true), () -> String.format( "'%s' must be at least 1mb character. Got: %s", BULK_FLASH_MAX_SIZE_OPTION.key(), originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) ); - validateOptions( + validate( config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), () -> String.format( "'%s' must be at least 1. Got: %s", @@ -131,7 +131,7 @@ private void validateOptions(Elasticsearch7Configuration config, Configuration o ); } - private static void validateOptions(boolean condition, Supplier message) { + private static void validate(boolean condition, Supplier message) { if (!condition) { throw new ValidationException(message.get()); } From d9ccbc7dd65393cda4afe4bda033b10bbdfb0a64 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 9 Jun 2020 10:47:15 +0800 Subject: [PATCH 096/207] [FLINK-16495][elasticsearch][table] Improve default flush strategy for new Elasticsearch sink for better out-of-box The default flush strategy for old Elasticsearch sink is no flush interval and 5MB buffered size and 1000 rows. The new default flush strategy for new Elasticsearch sink is '1s' flush interval and '1000' buffered rows and '2mb' buffered size. This closes #12536 --- .../table/ElasticsearchConfiguration.java | 19 +++++++----- .../table/ElasticsearchOptions.java | 6 ++-- .../util/NoOpFailureHandler.java | 9 ++++++ .../table/Elasticsearch6DynamicSink.java | 6 ++-- .../Elasticsearch6DynamicSinkFactory.java | 11 ++++--- .../elasticsearch6/ElasticsearchSink.java | 12 -------- .../Elasticsearch6DynamicSinkFactoryTest.java | 6 ++-- .../table/Elasticsearch6DynamicSinkTest.java | 29 +++++++++++++++++++ .../table/Elasticsearch7DynamicSink.java | 6 ++-- .../Elasticsearch7DynamicSinkFactory.java | 11 ++++--- .../Elasticsearch7DynamicSinkFactoryTest.java | 6 ++-- .../table/Elasticsearch7DynamicSinkTest.java | 29 +++++++++++++++++++ 12 files changed, 108 insertions(+), 42 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index 48b848cd..6de88928 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; @@ -81,16 +80,22 @@ public String getDocumentType() { return config.get(ElasticsearchOptions.DOCUMENT_TYPE_OPTION); } - public Optional getBulkFlushMaxActions() { - return config.getOptional(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + public int getBulkFlushMaxActions() { + int maxActions = config.get(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxActions == 0 ? -1 : maxActions; } - public Optional getBulkFlushMaxSize() { - return config.getOptional(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).map(MemorySize::getMebiBytes); + public long getBulkFlushMaxByteSize() { + long maxSize = config.get(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxSize == 0 ? -1 : maxSize; } - public Optional getBulkFlushInterval() { - return config.getOptional(BULK_FLUSH_INTERVAL_OPTION).map(Duration::toMillis); + public long getBulkFlushInterval() { + long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return interval == 0 ? -1 : interval; } public boolean isBulkFlushBackoffEnabled() { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index 176414d6..355f4550 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -84,17 +84,17 @@ public enum BackOffType { public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = ConfigOptions.key("sink.bulk-flush.max-actions") .intType() - .noDefaultValue() + .defaultValue(1000) .withDescription("Maximum number of actions to buffer for each bulk request."); public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = ConfigOptions.key("sink.bulk-flush.max-size") .memoryType() - .noDefaultValue() + .defaultValue(MemorySize.parse("2mb")) .withDescription("Maximum size of buffered actions per bulk request"); public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = ConfigOptions.key("sink.bulk-flush.interval") .durationType() - .noDefaultValue() + .defaultValue(Duration.ofSeconds(1)) .withDescription("Bulk flush interval"); public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = ConfigOptions.key("sink.bulk-flush.backoff.strategy") diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java index dfcb9ee8..4726dc11 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -37,4 +37,13 @@ public void onFailure(ActionRequest action, Throwable failure, int restStatusCod throw failure; } + @Override + public boolean equals(Object o) { + return o instanceof NoOpFailureHandler; + } + + @Override + public int hashCode() { + return NoOpFailureHandler.class.hashCode(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index 01b5f47f..4e9a8f85 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -128,9 +128,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { upsertFunction); builder.setFailureHandler(config.getFailureHandler()); - config.getBulkFlushMaxActions().ifPresent(builder::setBulkFlushMaxActions); - config.getBulkFlushMaxSize().ifPresent(builder::setBulkFlushMaxSizeMb); - config.getBulkFlushInterval().ifPresent(builder::setBulkFlushInterval); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index c5d9c893..071bbb6a 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -109,17 +109,20 @@ private void validate(Elasticsearch6Configuration config, Configuration original validate( config.getIndex().length() >= 1, () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); validate( - config.getBulkFlushMaxActions().map(maxActions -> maxActions >= 1).orElse(true), + maxActions == -1 || maxActions >= 1, () -> String.format( "'%s' must be at least 1 character. Got: %s", BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - config.getBulkFlushMaxActions().get()) + maxActions) ); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; validate( - config.getBulkFlushMaxSize().map(maxSize -> maxSize >= 1024 * 1024).orElse(true), + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), () -> String.format( - "'%s' must be at least 1mb character. Got: %s", + "'%s' must be in MB granularity. Got: %s", BULK_FLASH_MAX_SIZE_OPTION.key(), originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) ); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 484e6f6e..b8476138 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -104,10 +104,6 @@ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsea * @param numMaxActions the maxinum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { - Preconditions.checkArgument( - numMaxActions > 0, - "Max number of buffered actions must be larger than 0."); - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); } @@ -117,10 +113,6 @@ public void setBulkFlushMaxActions(int numMaxActions) { * @param maxSizeMb the maximum size of buffered actions, in mb. */ public void setBulkFlushMaxSizeMb(int maxSizeMb) { - Preconditions.checkArgument( - maxSizeMb > 0, - "Max size of buffered actions must be larger than 0."); - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); } @@ -130,10 +122,6 @@ public void setBulkFlushMaxSizeMb(int maxSizeMb) { * @param intervalMillis the bulk flush interval, in milliseconds. */ public void setBulkFlushInterval(long intervalMillis) { - Preconditions.checkArgument( - intervalMillis >= 0, - "Interval (in milliseconds) between each flush must be larger than or equal to 0."); - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 6d0878f2..143b712f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -102,7 +102,7 @@ public void validateWrongFlushSize() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-size' must be at least 1mb character. Got: 1024 bytes"); + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -142,7 +142,7 @@ public void validateWrongMaxActions() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1 character. Got: 0"); + "'sink.bulk-flush.max-actions' must be at least 1 character. Got: -2"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -151,7 +151,7 @@ public void validateWrongMaxActions() { .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "0") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") .build() ); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index 1708efc4..36d8f6df 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; @@ -41,6 +42,7 @@ import java.util.List; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; /** @@ -85,6 +87,33 @@ public void testBuilder() { verify(provider.sinkSpy).disableFlushOnCheckpoint(); } + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration(configuration, this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + private Configuration getConfig() { Configuration configuration = new Configuration(); configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 7aa52eaf..2213ce8a 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -128,9 +128,9 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { upsertFunction); builder.setFailureHandler(config.getFailureHandler()); - config.getBulkFlushMaxActions().ifPresent(builder::setBulkFlushMaxActions); - config.getBulkFlushMaxSize().ifPresent(builder::setBulkFlushMaxSizeMb); - config.getBulkFlushInterval().ifPresent(builder::setBulkFlushInterval); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 320c894b..7ee2cfbc 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -108,17 +108,20 @@ private void validate(Elasticsearch7Configuration config, Configuration original validate( config.getIndex().length() >= 1, () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); validate( - config.getBulkFlushMaxActions().map(maxActions -> maxActions >= 1).orElse(true), + maxActions == -1 || maxActions >= 1, () -> String.format( "'%s' must be at least 1 character. Got: %s", BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - config.getBulkFlushMaxActions().get()) + maxActions) ); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; validate( - config.getBulkFlushMaxSize().map(maxSize -> maxSize >= 1024 * 1024).orElse(true), + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), () -> String.format( - "'%s' must be at least 1mb character. Got: %s", + "'%s' must be in MB granularity. Got: %s", BULK_FLASH_MAX_SIZE_OPTION.key(), originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) ); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 4fe32142..71891637 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -99,7 +99,7 @@ public void validateWrongFlushSize() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-size' must be at least 1mb character. Got: 1024 bytes"); + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -137,7 +137,7 @@ public void validateWrongMaxActions() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1 character. Got: 0"); + "'sink.bulk-flush.max-actions' must be at least 1 character. Got: -2"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() @@ -145,7 +145,7 @@ public void validateWrongMaxActions() { .build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "0") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") .build() ); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index c972cee4..36c3135f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -24,6 +24,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; @@ -41,6 +42,7 @@ import java.util.List; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; /** @@ -85,6 +87,33 @@ public void testBuilder() { verify(provider.sinkSpy).disableFlushOnCheckpoint(); } + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration(configuration, this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + private Configuration getConfig() { Configuration configuration = new Configuration(); configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); From 491bb882284cd48fb811a1f0d7b18146e34f38fe Mon Sep 17 00:00:00 2001 From: Qishang Zhong Date: Fri, 12 Jun 2020 21:22:52 +0800 Subject: [PATCH 097/207] [FLINK-18277][elasticsearch] Fix the returned value of Elasticsearch6DynamicSink#asSummaryString() This closes #12633 --- .../elasticsearch/table/Elasticsearch6DynamicSink.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index 4e9a8f85..b7fd44d4 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -157,7 +157,7 @@ public DynamicTableSink copy() { @Override public String asSummaryString() { - return "Elasticsearch7"; + return "Elasticsearch6"; } /** From e7e46c13b960af59f67b3abc1a05227f272ef702 Mon Sep 17 00:00:00 2001 From: Yun Wang Date: Thu, 11 Jun 2020 17:11:54 -0700 Subject: [PATCH 098/207] [FLINK-17623][elasticsearch] Support user resource cleanup in ElasticsearchSinkFunction This closes #12619 --- .../elasticsearch/ElasticsearchSinkBase.java | 1 + .../ElasticsearchSinkFunction.java | 7 +++- .../ElasticsearchSinkBaseTest.java | 35 +++++++++++++++++++ 3 files changed, 42 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 91beaf06..28110083 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -329,6 +329,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { @Override public void close() throws Exception { + elasticsearchSinkFunction.close(); if (bulkProcessor != null) { bulkProcessor.close(); bulkProcessor = null; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index 8ff6babf..d023a448 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -63,7 +63,12 @@ public interface ElasticsearchSinkFunction extends Serializable, Function { /** * Initialization method for the function. It is called once before the actual working process methods. */ - default void open() {} + default void open() throws Exception {} + + /** + * Tear-down method for the function. It is called when the sink closes. + */ + default void close () throws Exception {} /** * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 0648ed91..7e55c922 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CheckedThread; import org.apache.flink.core.testutils.MultiShotLatch; import org.apache.flink.streaming.api.operators.StreamSink; @@ -412,6 +413,19 @@ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Excepti testHarness.close(); } + @Test + public void testOpenAndCloseInSinkFunction() throws Exception { + SimpleClosableSinkFunction sinkFunction = new SimpleClosableSinkFunction<>(); + final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( + new HashMap<>(), sinkFunction, new DummyRetryFailureHandler()); + + sink.open(mock(Configuration.class)); + sink.close(); + + Assert.assertTrue(sinkFunction.openCalled); + Assert.assertTrue(sinkFunction.closeCalled); + } + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 5051907841570096991L; @@ -586,6 +600,27 @@ public void process(String element, RuntimeContext ctx, RequestIndexer indexer) } } + private static class SimpleClosableSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1872065917794006848L; + + private boolean openCalled; + private boolean closeCalled; + + @Override + public void open() { + openCalled = true; + } + + @Override + public void close() { + closeCalled = true; + } + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {} + } + private static class DummyRetryFailureHandler implements ActionRequestFailureHandler { private static final long serialVersionUID = 5400023700099200745L; From 87a9ae1e2a3aab628acde465f79ecf36c5ecb55e Mon Sep 17 00:00:00 2001 From: Shengkai <33114724+fsk119@users.noreply.github.com> Date: Wed, 17 Jun 2020 21:32:10 +0800 Subject: [PATCH 099/207] [FLINK-18299][json] Fix the non SQL standard timestamp format in JSON format The current timestamp format in JSON format is not SQL standard which uses RFC-3339. This commit changes the default behavior to parse/generate timestamp using SQL standard. Besides, it introduces an option "json.timestamp-format.standard" to have the ability to fallback to ISO standard. This closes #12661 --- .../table/Elasticsearch6DynamicSinkITCase.java | 12 ++++++------ .../table/Elasticsearch7DynamicSinkITCase.java | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 41a67f62..26cf90aa 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -107,12 +107,12 @@ public void testWritingDocuments() throws Exception { Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(response, equalTo(expectedMap)); } @@ -165,12 +165,12 @@ public void testWritingDocumentsFromTableApi() throws Exception { .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(response, equalTo(expectedMap)); } @@ -238,12 +238,12 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { Map result = hits.getAt(0).getSourceAsMap(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(result, equalTo(expectedMap)); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index dbd1ff90..7f41eb78 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -105,12 +105,12 @@ public void testWritingDocuments() throws Exception { Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(response, equalTo(expectedMap)); } @@ -159,12 +159,12 @@ public void testWritingDocumentsFromTableApi() throws Exception { Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(response, equalTo(expectedMap)); } @@ -230,12 +230,12 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { Map result = hits.getAt(0).getSourceAsMap(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12Z"); + expectedMap.put("b", "00:00:12"); expectedMap.put("c", "ABCDE"); expectedMap.put("d", 12.12d); expectedMap.put("e", 2); expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12T12:12:12Z"); + expectedMap.put("g", "2012-12-12 12:12:12"); assertThat(result, equalTo(expectedMap)); } From b7dcecbb6710dba603280c4dfa381d8ead39b18a Mon Sep 17 00:00:00 2001 From: liuyongvs Date: Wed, 10 Jun 2020 17:13:33 +0800 Subject: [PATCH 100/207] [FLINK-18236] fix es connector test ElasticsearchSinkTestBase.runElasticsearchSink* verify not right. --- .../elasticsearch/ElasticsearchSinkTestBase.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 35bdca7b..540f4316 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -53,33 +53,31 @@ public abstract class ElasticsearchSinkTestBase exte * Tests that the Elasticsearch sink works properly with json. */ public void runElasticsearchSinkTest() throws Exception { - runElasticSearchSinkTest(SourceSinkDataTestKit::getJsonSinkFunction); + runElasticSearchSinkTest("elasticsearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction); } /** * Tests that the Elasticsearch sink works properly with cbor. */ public void runElasticsearchSinkCborTest() throws Exception { - runElasticSearchSinkTest(SourceSinkDataTestKit::getCborSinkFunction); + runElasticSearchSinkTest("elasticsearch-sink-test-cbor-index", SourceSinkDataTestKit::getCborSinkFunction); } /** * Tests that the Elasticsearch sink works properly with smile. */ public void runElasticsearchSinkSmileTest() throws Exception { - runElasticSearchSinkTest(SourceSinkDataTestKit::getSmileSinkFunction); + runElasticSearchSinkTest("elasticsearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction); } /** * Tests that the Elasticsearch sink works properly with yaml. */ public void runElasticsearchSinkYamlTest() throws Exception { - runElasticSearchSinkTest(SourceSinkDataTestKit::getYamlSinkFunction); + runElasticSearchSinkTest("elasticsearch-sink-test-yaml-index", SourceSinkDataTestKit::getYamlSinkFunction); } - private void runElasticSearchSinkTest(Function>> functionFactory) throws Exception { - final String index = "elasticsearch-sink-test-index"; - + private void runElasticSearchSinkTest(String index, Function>> functionFactory) throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); From 0ad4a2609743ff436e56101738f117b10b8a01cd Mon Sep 17 00:00:00 2001 From: rinkako Date: Thu, 18 Jun 2020 19:56:17 +0800 Subject: [PATCH 101/207] [FLINK-18359] Log failures in handler instead of ElasticsearchSinkBase This allows more control for the handler whether or not to log error messages. In some cases, users know that they will get a lot of failures, for example when back-filling existing data in ES. For those, you don't want your log flooded with ERROR messages. --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 8 -------- .../connectors/elasticsearch/util/NoOpFailureHandler.java | 5 +++++ .../util/RetryRejectedExecutionFailureHandler.java | 5 +++++ 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 28110083..d19fba67 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -39,8 +39,6 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.RestStatus; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.Serializable; import java.util.HashMap; @@ -71,8 +69,6 @@ public abstract class ElasticsearchSinkBase extends private static final long serialVersionUID = -1007596293618451942L; - private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkBase.class); - // ------------------------------------------------------------------------ // Internal bulk processor configuration // ------------------------------------------------------------------------ @@ -408,8 +404,6 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon itemResponse = response.getItems()[i]; failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); if (failure != null) { - LOG.error("Failed Elasticsearch item request: {}", itemResponse.getFailureMessage(), failure); - restStatus = itemResponse.getFailure().getStatus(); actionRequest = request.requests().get(i); if (restStatus == null) { @@ -441,8 +435,6 @@ public void afterBulk(long executionId, BulkRequest request, BulkResponse respon @Override public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - LOG.error("Failed Elasticsearch bulk request: {}", failure.getMessage(), failure); - try { for (DocWriteRequest writeRequest : request.requests()) { if (writeRequest instanceof ActionRequest) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java index 4726dc11..c076fc8e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -22,6 +22,8 @@ import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.elasticsearch.action.ActionRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. @@ -31,8 +33,11 @@ public class NoOpFailureHandler implements ActionRequestFailureHandler { private static final long serialVersionUID = 737941343410827885L; + private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class); + @Override public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); // simply fail the sink throw failure; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index ca710cb7..98b58f94 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -25,6 +25,8 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary @@ -36,8 +38,11 @@ public class RetryRejectedExecutionFailureHandler implements ActionRequestFailur private static final long serialVersionUID = -7423562912824511906L; + private static final Logger LOG = LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class); + @Override public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { indexer.add(action); } else { From 26b3ae890f9ba38c2e540d7f0280e2457cd63ef7 Mon Sep 17 00:00:00 2001 From: yushengnan Date: Fri, 19 Jun 2020 21:45:52 +0800 Subject: [PATCH 102/207] [FLINK-14938] Use ConcurrentLinkedQueue in BufferingNoOpRequestIndexer This solves the problem of concurrent modification when re-adding ES index requests from a failure handler. --- .../elasticsearch/BufferingNoOpRequestIndexer.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java index e639b820..07341da8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java @@ -27,9 +27,8 @@ import javax.annotation.concurrent.NotThreadSafe; -import java.util.ArrayList; import java.util.Collections; -import java.util.List; +import java.util.concurrent.ConcurrentLinkedQueue; /** * Implementation of a {@link RequestIndexer} that buffers {@link ActionRequest ActionRequests} @@ -39,10 +38,10 @@ @NotThreadSafe class BufferingNoOpRequestIndexer implements RequestIndexer { - private List bufferedRequests; + private ConcurrentLinkedQueue bufferedRequests; BufferingNoOpRequestIndexer() { - this.bufferedRequests = new ArrayList<>(10); + this.bufferedRequests = new ConcurrentLinkedQueue(); } @Override From dd86d981173ae3d89fd7533fe41616111b057581 Mon Sep 17 00:00:00 2001 From: liuyongvs Date: Mon, 29 Jun 2020 16:56:34 +0800 Subject: [PATCH 103/207] [hotfix][elasticsearch] Fix validation message for 'sink.bulk-flush.max-actions' of Elasticsearch This closes #12790 --- .../elasticsearch/table/Elasticsearch6DynamicSinkFactory.java | 2 +- .../table/Elasticsearch6DynamicSinkFactoryTest.java | 2 +- .../elasticsearch/table/Elasticsearch7DynamicSinkFactory.java | 2 +- .../table/Elasticsearch7DynamicSinkFactoryTest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 071bbb6a..70b8d38b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -113,7 +113,7 @@ private void validate(Elasticsearch6Configuration config, Configuration original validate( maxActions == -1 || maxActions >= 1, () -> String.format( - "'%s' must be at least 1 character. Got: %s", + "'%s' must be at least 1. Got: %s", BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions) ); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 143b712f..b213c5d6 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -142,7 +142,7 @@ public void validateWrongMaxActions() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1 character. Got: -2"); + "'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 7ee2cfbc..be081880 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -112,7 +112,7 @@ private void validate(Elasticsearch7Configuration config, Configuration original validate( maxActions == -1 || maxActions >= 1, () -> String.format( - "'%s' must be at least 1 character. Got: %s", + "'%s' must be at least 1. Got: %s", BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions) ); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 71891637..9db718c1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -137,7 +137,7 @@ public void validateWrongMaxActions() { thrown.expect(ValidationException.class); thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1 character. Got: -2"); + "'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() .withSchema(TableSchema.builder() From 35703c9be88454f666b1a7a8aada855ad62d125a Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Fri, 10 Jul 2020 16:41:08 +0800 Subject: [PATCH 104/207] [FLINK-18361][es][table] Support username and password options for new Elasticsearch connector Co-authored-by: zhisheng17 This closes #12715 --- .../table/ElasticsearchConfiguration.java | 10 +++ .../table/ElasticsearchOptions.java | 10 +++ .../table/Elasticsearch6DynamicSink.java | 64 ++++++++++++++++++- .../Elasticsearch6DynamicSinkFactory.java | 18 +++++- .../Elasticsearch6DynamicSinkFactoryTest.java | 21 ++++++ .../table/Elasticsearch6DynamicSinkTest.java | 31 +++++++++ .../table/Elasticsearch7DynamicSink.java | 64 ++++++++++++++++++- .../Elasticsearch7DynamicSinkFactory.java | 18 +++++- .../Elasticsearch7DynamicSinkFactoryTest.java | 20 ++++++ .../table/Elasticsearch7DynamicSinkTest.java | 31 +++++++++ 10 files changed, 283 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index 6de88928..25b5d6b9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -37,6 +37,8 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; /** * Accessor methods to elasticsearch options. @@ -98,6 +100,14 @@ public long getBulkFlushInterval() { return interval == 0 ? -1 : interval; } + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + public boolean isBulkFlushBackoffEnabled() { return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) != ElasticsearchOptions.BackOffType.DISABLED; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index 355f4550..b07190c5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -59,6 +59,16 @@ public enum BackOffType { .stringType() .noDefaultValue() .withDescription("Elasticsearch document type."); + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); public static final ConfigOption KEY_DELIMITER_OPTION = ConfigOptions.key("document-id.key-delimiter") .stringType() diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index b7fd44d4..8197817e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -30,8 +30,13 @@ import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; @@ -138,7 +143,14 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { // we must overwrite the default factory which is defined with a lambda because of a bug // in shading lambda serialization shading see FLINK-18006 - builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory(new AuthRestClientFactory(config.getPathPrefix().orElse(null), config.getUsername().get(), config.getPassword().get())); + } else { + builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } final ElasticsearchSink sink = builder.build(); @@ -197,6 +209,56 @@ public int hashCode() { } } + /** + * Serializable {@link RestClientFactory} used by the sink which enable authentication. + */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory(@Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback(httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) && + Objects.equals(username, that.username) && + Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, username, password); + } + } + /** * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. */ diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 70b8d38b..5fb57697 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -31,6 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; import java.util.Set; import java.util.function.Supplier; @@ -52,6 +53,8 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; /** * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. @@ -75,7 +78,9 @@ public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_DELAY_OPTION, CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, - FORMAT_OPTION + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION ).collect(Collectors.toSet()); @Override @@ -133,6 +138,17 @@ private void validate(Elasticsearch6Configuration config, Configuration original BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), config.getBulkFlushBackoffRetries().get()) ); + if (config.getUsername().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse("") + )); + } } private static void validate(boolean condition, Supplier message) { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index b213c5d6..3d90c890 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -204,4 +204,25 @@ public void validatePrimaryKeyOnIllegalColumn() { .build() ); } + + @Test + public void validateWrongCredential() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + .build() + ); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index 36d8f6df..0e54ed54 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -60,6 +60,8 @@ public class Elasticsearch6DynamicSinkTest { private static final String SCHEMA = "https"; private static final String INDEX = "MyIndex"; private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; @Test public void testBuilder() { @@ -114,6 +116,35 @@ public void testDefaultConfig() { verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); } + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration(configuration, this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.AuthRestClientFactory(null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + private Configuration getConfig() { Configuration configuration = new Configuration(); configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 2213ce8a..52efbe1c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -30,8 +30,13 @@ import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; @@ -138,7 +143,14 @@ public SinkFunctionProvider getSinkRuntimeProvider(Context context) { // we must overwrite the default factory which is defined with a lambda because of a bug // in shading lambda serialization shading see FLINK-18006 - builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory(new AuthRestClientFactory(config.getPathPrefix().orElse(null), config.getUsername().get(), config.getPassword().get())); + } else { + builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } final ElasticsearchSink sink = builder.build(); @@ -197,6 +209,56 @@ public int hashCode() { } } + /** + * Serializable {@link RestClientFactory} used by the sink which enable authentication. + */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory(@Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback(httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) && + Objects.equals(username, that.username) && + Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, password, username); + } + } + /** * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. */ diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index be081880..a68a0efa 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -31,6 +31,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; import java.util.Set; import java.util.function.Supplier; @@ -51,6 +52,8 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; /** * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. @@ -73,7 +76,9 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_DELAY_OPTION, CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, - FORMAT_OPTION + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION ).collect(Collectors.toSet()); @Override @@ -132,6 +137,17 @@ private void validate(Elasticsearch7Configuration config, Configuration original BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), config.getBulkFlushBackoffRetries().get()) ); + if (config.getUsername().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse("") + )); + } } private static void validate(boolean condition, Supplier message) { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 9db718c1..d34e819c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -197,4 +197,24 @@ public void validatePrimaryKeyOnIllegalColumn() { .build() ); } + + @Test + public void validateWrongCredential() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder() + .field("a", DataTypes.TIME()) + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + .build() + ); + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index 36c3135f..7e18d64c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -60,6 +60,8 @@ public class Elasticsearch7DynamicSinkTest { private static final String SCHEMA = "https"; private static final String INDEX = "MyIndex"; private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; @Test public void testBuilder() { @@ -114,6 +116,35 @@ public void testDefaultConfig() { verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); } + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration(configuration, this.getClass().getClassLoader()), + schema, + provider + ); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.AuthRestClientFactory(null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + private Configuration getConfig() { Configuration configuration = new Configuration(); configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); From 4ccce90321b7c478f99fb33924f7dbadf4239a10 Mon Sep 17 00:00:00 2001 From: Leonard Xu Date: Tue, 14 Jul 2020 10:45:02 +0800 Subject: [PATCH 105/207] [FLINK-18585][elasticsearch] Fix dynamic index doesn't work in new elasticsearch table sink This closes #12886 --- .../table/IndexGeneratorFactory.java | 2 +- .../table/RowElasticsearchSinkFunction.java | 5 +++ .../Elasticsearch6DynamicSinkITCase.java | 37 +++++++++++++++++++ .../Elasticsearch7DynamicSinkITCase.java | 37 +++++++++++++++++++ 4 files changed, 80 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java index e60be723..692b1fe3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -55,7 +55,7 @@ * convert a field value of TIMESTAMP/DATE/TIME type into the format specified by date_format_string. The * date_format_string is compatible with {@link java.text.SimpleDateFormat}. For example, if the option * value is 'myusers_{log_ts|yyyy-MM-dd}', then a record with log_ts field value 2020-03-27 12:25:55 will - * be written into "myusers-2020-03-27" index. + * be written into "myusers_2020-03-27" index. */ @Internal final class IndexGeneratorFactory { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java index 4eaba480..76701d8b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -68,6 +68,11 @@ public RowElasticsearchSinkFunction( this.createKey = Preconditions.checkNotNull(createKey); } + @Override + public void open() { + indexGenerator.open(); + } + @Override public void process( RowData element, diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 26cf90aa..b306b34b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -247,6 +247,43 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { assertThat(result, equalTo(expectedMap)); } + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + String myType = "MyType"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + private static class MockContext implements DynamicTableSink.Context { @Override public boolean isBounded() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 7f41eb78..483a7838 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -239,6 +239,43 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { assertThat(result, equalTo(expectedMap)); } + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql("CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); + + tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .getJobClient() + .get() + .getJobExecutionResult(this.getClass().getClassLoader()) + .get(); + + Client client = elasticsearchResource.getClient(); + Map response = client.get(new GetRequest("dynamic-index-2012-12-12", "1")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + private static class MockContext implements DynamicTableSink.Context { @Override public boolean isBounded() { From 65431b148594c5b9fada0d3b26980028115ad640 Mon Sep 17 00:00:00 2001 From: Yangze Guo Date: Tue, 14 Jul 2020 19:31:26 +0800 Subject: [PATCH 106/207] [FLINK-18583][elasticsearch] Fix ElasticSearch6 sink uses index as document id This closes #12890 --- .../table/Elasticsearch6DynamicSink.java | 2 +- .../Elasticsearch6DynamicSinkITCase.java | 50 +++++++++++++------ .../Elasticsearch7DynamicSinkITCase.java | 50 +++++++++++++------ 3 files changed, 73 insertions(+), 29 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index 8197817e..d4e5866c 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -282,7 +282,7 @@ public IndexRequest createIndexRequest( String key, XContentType contentType, byte[] document) { - return new IndexRequest(index, docType, index) + return new IndexRequest(index, docType, key) .source(document, contentType); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index b306b34b..8a7c1805 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -47,6 +47,7 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; @@ -208,7 +209,15 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { 12.12f, (byte) 2, LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")) + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13")) ).executeInsert("esTable") .getJobClient() .get() @@ -226,25 +235,38 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { .execute() .actionGet() .getHits(); - if (hits.getTotalHits() < 1) { + if (hits.getTotalHits() < 2) { Thread.sleep(200); } - } while (hits.getTotalHits() < 1 && deadline.hasTimeLeft()); + } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); - if (hits.getTotalHits() < 1) { + if (hits.getTotalHits() < 2) { throw new AssertionError("Could not retrieve results from Elasticsearch."); } - Map result = hits.getAt(0).getSourceAsMap(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(result, equalTo(expectedMap)); + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); } @Test diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 483a7838..b011e5bf 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -47,6 +47,7 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; @@ -200,7 +201,15 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { 12.12f, (byte) 2, LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")) + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13")) ).executeInsert("esTable") .getJobClient() .get() @@ -218,25 +227,38 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { .execute() .actionGet() .getHits(); - if (hits.getTotalHits().value < 1) { + if (hits.getTotalHits().value < 2) { Thread.sleep(200); } - } while (hits.getTotalHits().value < 1 && deadline.hasTimeLeft()); + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); - if (hits.getTotalHits().value < 1) { + if (hits.getTotalHits().value < 2) { throw new AssertionError("Could not retrieve results from Elasticsearch."); } - Map result = hits.getAt(0).getSourceAsMap(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(result, equalTo(expectedMap)); + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); } @Test From 83f1634c71c27af7d9e68d7b579da77cee08a98b Mon Sep 17 00:00:00 2001 From: Niels Basjes Date: Wed, 22 Jul 2020 11:26:03 +0200 Subject: [PATCH 107/207] [FLINK-18607][build] Give the maven module a human readable name This closes #12907 --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 8ada7132..fc37c05c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -31,7 +31,7 @@ under the License. flink-connector-elasticsearch-base_${scala.binary.version} - flink-connector-elasticsearch-base + Flink : Connectors : Elasticsearch base jar diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 52f9cf6c..e796af83 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -31,7 +31,7 @@ under the License. flink-connector-elasticsearch6_${scala.binary.version} - flink-connector-elasticsearch6 + Flink : Connectors : Elasticsearch 6 jar diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index d0385e5f..f1b9e822 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -31,7 +31,7 @@ under the License. flink-connector-elasticsearch7_${scala.binary.version} - flink-connector-elasticsearch7 + Flink : Connectors : Elasticsearch 7 jar diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 97166ac6..efd1d306 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -31,7 +31,7 @@ under the License. flink-sql-connector-elasticsearch6_${scala.binary.version} - flink-sql-connector-elasticsearch6 + Flink : Connectors : SQL : Elasticsearch 6 jar diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 0b165461..0ad181bf 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -31,7 +31,7 @@ under the License. flink-sql-connector-elasticsearch7_${scala.binary.version} - flink-sql-connector-elasticsearch7 + Flink : Connectors : SQL : Elasticsearch 7 jar From 42f39df3a1921e3b3cde907d85eb60590a8290ed Mon Sep 17 00:00:00 2001 From: Brandon Date: Mon, 17 Aug 2020 16:56:12 +0800 Subject: [PATCH 108/207] [hoxfix] Fix various typos --- .../connectors/elasticsearch/ElasticsearchApiCallBridge.java | 2 +- .../connectors/elasticsearch/table/ElasticsearchOptions.java | 2 +- .../streaming/connectors/elasticsearch6/ElasticsearchSink.java | 2 +- .../streaming/connectors/elasticsearch7/ElasticsearchSink.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 7bbbe7b9..5a1ecf13 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -56,7 +56,7 @@ public interface ElasticsearchApiCallBridge extends Ser * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. * * @param client the Elasticsearch client. - * @param listener the bulk processor listender. + * @param listener the bulk processor listener. * @return the bulk processor builder. */ BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index b07190c5..9cd781a5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -48,7 +48,7 @@ public enum BackOffType { .stringType() .asList() .noDefaultValue() - .withDescription("Elasticseatch hosts to connect to."); + .withDescription("Elasticsearch hosts to connect to."); public static final ConfigOption INDEX_OPTION = ConfigOptions.key("index") .stringType() diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index b8476138..c9465a22 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -101,7 +101,7 @@ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsea /** * Sets the maximum number of actions to buffer for each bulk request. * - * @param numMaxActions the maxinum number of actions to buffer per bulk request. + * @param numMaxActions the maximum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java index 5b874e29..5aae8954 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -101,7 +101,7 @@ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsea /** * Sets the maximum number of actions to buffer for each bulk request. * - * @param numMaxActions the maxinum number of actions to buffer per bulk request. + * @param numMaxActions the maximum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { Preconditions.checkArgument( From fae14ee6a112dd6c5d8d609b32bfa2838c8c0e1d Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 25 Aug 2020 10:25:17 +0200 Subject: [PATCH 109/207] [FLINK-17159] Harden ElasticsearchSinkITCase Before, it could happen that the embedded node is not ready. Now we wait for nodes/data nodes to be live before returning from the initialization method. --- .../testutils/ElasticsearchResource.java | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java index 6f185d30..68e0fe8a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java @@ -21,12 +21,21 @@ import org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironment; import org.apache.flink.util.InstantiationUtil; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; +import org.elasticsearch.client.ClusterAdminClient; +import org.elasticsearch.common.unit.TimeValue; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.junit.Assert.assertThat; + /** * A resource that starts an embedded elasticsearch cluster. */ @@ -55,6 +64,27 @@ protected void before() throws Throwable { tempFolder.create(); embeddedNodeEnv.start(tempFolder.newFolder(), clusterName); + + waitForCluster(); + } + + /** + * Blocks until the cluster is ready and data nodes/nodes are live. + */ + private void waitForCluster() { + AdminClient adminClient = embeddedNodeEnv.getClient().admin(); + ClusterAdminClient clusterAdminClient = adminClient.cluster(); + + ClusterHealthRequestBuilder requestBuilder = clusterAdminClient.prepareHealth("_all"); + requestBuilder = requestBuilder.setTimeout(TimeValue.timeValueSeconds(120)); + + ActionFuture healthFuture = + clusterAdminClient.health(requestBuilder.request()); + + ClusterHealthResponse health = healthFuture.actionGet(TimeValue.timeValueSeconds(120)); + + assertThat(health.getNumberOfNodes(), greaterThanOrEqualTo(1)); + assertThat(health.getNumberOfDataNodes(), greaterThanOrEqualTo(1)); } @Override From c91bbfbaaa8027d2e61eb1edb78d7ee65ddafcab Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Wed, 17 Jun 2020 14:23:46 +0800 Subject: [PATCH 110/207] [FLINK-18337][table] Introduce TableResult#await method to block until data is ready or job finished This closes #12688. --- .../table/Elasticsearch6DynamicSinkITCase.java | 18 +++--------------- .../table/Elasticsearch7DynamicSinkITCase.java | 18 +++--------------- 2 files changed, 6 insertions(+), 30 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 8a7c1805..ce57fadb 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -154,11 +154,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { (byte) 2, LocalDate.ofEpochDay(12345), LocalDateTime.parse("2012-12-12T12:12:12")) - ).executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + ).executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) @@ -218,11 +214,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { (byte) 4, LocalDate.ofEpochDay(12345), LocalDateTime.parse("2013-12-12T13:13:13")) - ).executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + ).executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); @@ -292,11 +284,7 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { ")"); tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + .executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); Map response = client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")).actionGet().getSource(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index b011e5bf..d2964f55 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -150,11 +150,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { (byte) 2, LocalDate.ofEpochDay(12345), LocalDateTime.parse("2012-12-12T12:12:12")) - ).executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + ).executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); @@ -210,11 +206,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { (byte) 4, LocalDate.ofEpochDay(12345), LocalDateTime.parse("2013-12-12T13:13:13")) - ).executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + ).executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); @@ -282,11 +274,7 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { ")"); tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .getJobClient() - .get() - .getJobExecutionResult(this.getClass().getClassLoader()) - .get(); + .executeInsert("esTable").await(); Client client = elasticsearchResource.getClient(); Map response = client.get(new GetRequest("dynamic-index-2012-12-12", "1")) From 943a7e778261d0b3a736b9c65f890d791954de62 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 22 Sep 2020 10:42:47 +0200 Subject: [PATCH 111/207] [FLINK-17159] Add sanity check to ES6 ElasticsearchSinkITCase We now try and ping the cluster in a @Before method just like the normal ES sink would. --- .../elasticsearch6/ElasticsearchSinkITCase.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 27903df8..4a1d7e82 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -24,9 +24,13 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; +import org.junit.Before; import org.junit.Test; +import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -35,6 +39,15 @@ */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + @Before + public void ensureClusterIsUp() throws IOException { + RestClientBuilder builder = RestClient.builder(HttpHost.create("http://127.0.0.1:9200")); + RestHighLevelClient client = new RestHighLevelClient(builder); + if (!client.ping()) { + throw new RuntimeException("Cannot ping cluster!"); + } + } + @Test public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); From 23fa4662982953bd5146de6a3b606165d662fa1d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 22 Sep 2020 11:03:41 +0200 Subject: [PATCH 112/207] [FLINK-19243][elasticsearch] Bump snakeyaml to 1.27 --- .../flink-connector-elasticsearch-base/pom.xml | 11 +++++++++++ .../flink-connector-elasticsearch6/pom.xml | 7 +++++++ .../flink-connector-elasticsearch7/pom.xml | 7 +++++++ 3 files changed, 25 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index fc37c05c..e43f256f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -158,6 +158,17 @@ under the License.
+ + + + + org.yaml + snakeyaml + 1.27 + + + + diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index e796af83..946cc691 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -64,6 +64,13 @@ under the License. + + + org.yaml + snakeyaml + 1.27 + + org.elasticsearch.client diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index f1b9e822..c6ebfac8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -64,6 +64,13 @@ under the License. + + + org.yaml + snakeyaml + 1.27 + + org.elasticsearch.client From 79b8b497d718ebfba20129865505766228ae705c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 22 Sep 2020 16:39:32 +0200 Subject: [PATCH 113/207] [FLINK-19341][table] Update all API related methods to FLIP-107 This updates all API related classes to support the concept of a metadata column. It considers all location until planner level. - Updates TableColumn to a hierarchy of 3 column types. - Updates TableSchemaUtils and all related locations that rely on computed column but have to deal with metadata column as well now. This closes #13480. This updates all API related classes to support the concept of a metadata column. It considers all location until planner level. --- .../elasticsearch/ElasticsearchUpsertTableSinkBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index defc804b..10ed5a40 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -116,7 +116,7 @@ public ElasticsearchUpsertTableSinkBase( RequestFactory requestFactory) { this.isAppendOnly = isAppendOnly; - this.schema = TableSchemaUtils.checkNoGeneratedColumns(schema); + this.schema = TableSchemaUtils.checkOnlyPhysicalColumns(schema); this.hosts = Preconditions.checkNotNull(hosts); this.index = Preconditions.checkNotNull(index); this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); From 17d83e669af554488bf2ed9117a9edf5f06d3d2f Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Tue, 6 Oct 2020 11:17:01 +0200 Subject: [PATCH 114/207] [FLINK-19485] Add the Transformation.getInputs() method This method is added to unify the way current transformations expose their input transformations to the StreamGraphGenerator. This will serve as a building block in the following commits in order to unify the way Transformations are translated. --- .../Elasticsearch6UpsertTableSinkFactoryTest.java | 5 +++++ .../Elasticsearch7UpsertTableSinkFactoryTest.java | 5 +++++ 2 files changed, 10 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index a5b5ef18..582a1d85 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -217,5 +217,10 @@ public TransformationMock(String name, TypeInformation> out public Collection> getTransitivePredecessors() { return null; } + + @Override + public List> getInputs() { + return Collections.emptyList(); + } } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index c24e3279..b77d1032 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -215,5 +215,10 @@ public TransformationMock(String name, TypeInformation> out public Collection> getTransitivePredecessors() { return null; } + + @Override + public List> getInputs() { + return Collections.emptyList(); + } } } From 4710b2388f496a0998faf47d3465e2f19255ef17 Mon Sep 17 00:00:00 2001 From: Kostas Kloudas Date: Fri, 9 Oct 2020 14:49:56 +0200 Subject: [PATCH 115/207] [hotfix] Make Transformation.getTransitivePredecessors() return a list --- .../Elasticsearch6UpsertTableSinkFactoryTest.java | 3 +-- .../Elasticsearch7UpsertTableSinkFactoryTest.java | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 582a1d85..29e9d491 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -47,7 +47,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.junit.Test; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -214,7 +213,7 @@ public TransformationMock(String name, TypeInformation> out } @Override - public Collection> getTransitivePredecessors() { + public List> getTransitivePredecessors() { return null; } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index b77d1032..099d6ef9 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -46,7 +46,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.junit.Test; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -212,7 +211,7 @@ public TransformationMock(String name, TypeInformation> out } @Override - public Collection> getTransitivePredecessors() { + public List> getTransitivePredecessors() { return null; } From b952fa7e558fc548c104c42001aff502fb72332f Mon Sep 17 00:00:00 2001 From: Rui Li Date: Fri, 21 Aug 2020 20:17:33 +0800 Subject: [PATCH 116/207] [FLINK-18999][table-common][table-planner-blink] Add isTemporary flag to table factory context --- .../connectors/elasticsearch/table/TestContext.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java index e4978fc1..32db4d35 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -69,6 +69,11 @@ public ReadableConfig getConfiguration() { public ClassLoader getClassLoader() { return TestContext.class.getClassLoader(); } + + @Override + public boolean isTemporary() { + return false; + } }; } From 87ad26a4b91a0113aa886125114a940b80f3a097 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Wed, 14 Oct 2020 16:41:47 +0200 Subject: [PATCH 117/207] [FLINK-19459] Add snakeyaml to dependencyManagement --- .../flink-connector-elasticsearch-base/pom.xml | 11 ----------- .../flink-connector-elasticsearch6/pom.xml | 7 ------- .../flink-connector-elasticsearch7/pom.xml | 7 ------- 3 files changed, 25 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index e43f256f..fc37c05c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -158,17 +158,6 @@ under the License. - - - - - org.yaml - snakeyaml - 1.27 - - - - diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 946cc691..e796af83 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -64,13 +64,6 @@ under the License. - - - org.yaml - snakeyaml - 1.27 - - org.elasticsearch.client diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index c6ebfac8..f1b9e822 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -64,13 +64,6 @@ under the License. - - - org.yaml - snakeyaml - 1.27 - - org.elasticsearch.client From 4e14eb30f38e18034e0494d08402ed3b546a9884 Mon Sep 17 00:00:00 2001 From: Robert Metzger Date: Fri, 23 Oct 2020 15:00:02 +0200 Subject: [PATCH 118/207] [FLINK-19781] Upgrade commons-codec to 1.13 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 542a1f6c..b77afb5a 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -10,7 +10,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 -- commons-codec:commons-codec:1.10 +- commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 - org.apache.httpcomponents:httpclient:4.5.3 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index ea7772dc..5a8708ec 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -12,7 +12,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 - com.github.spullara.mustache.java:compiler:0.9.6 -- commons-codec:commons-codec:1.10 +- commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.4 - org.apache.httpcomponents:httpclient:4.5.3 From 84aeb1333d10ad7ca0431d865e507ff41fc5397c Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Tue, 17 Nov 2020 11:05:20 +0100 Subject: [PATCH 119/207] [FLINK-17159] Use testcontainers for Elasticsearch ITCases This should harden the tests because we now use random ports for the ES nodes. I have a suspicion that we had clashes before, which were causing test instability. Additionally, this allows us to get rid of quite some of our own code for setting up an ES environment. Once we drop ES5 we can additionally drop ElasticsearchResource and EmbeddedElasticsearchNodeEnvironment. --- .../ElasticsearchSinkTestBase.java | 22 +++--- .../flink-connector-elasticsearch6/pom.xml | 7 ++ ...eddedElasticsearchNodeEnvironmentImpl.java | 79 ------------------- .../Elasticsearch6DynamicSinkITCase.java | 37 ++++++--- .../ElasticsearchSinkITCase.java | 50 ++++++++---- .../flink-connector-elasticsearch7/pom.xml | 7 ++ ...eddedElasticsearchNodeEnvironmentImpl.java | 79 ------------------- .../Elasticsearch7DynamicSinkITCase.java | 37 ++++++--- .../ElasticsearchSinkITCase.java | 41 ++++++++-- 9 files changed, 147 insertions(+), 212 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 540f4316..034161ec 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -21,12 +21,10 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.flink.test.util.AbstractTestBase; import org.elasticsearch.client.Client; -import org.junit.ClassRule; import java.util.Collections; import java.util.HashMap; @@ -44,10 +42,11 @@ */ public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { - protected static final String CLUSTER_NAME = "test-cluster"; + // It's not good that we're using a Client here instead of a Rest Client but we need this + // for compatibility with ES 5.3.x. As soon as we drop that we can use RestClient here. + protected abstract Client getClient(); - @ClassRule - public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource(CLUSTER_NAME); + protected abstract String getClusterName(); /** * Tests that the Elasticsearch sink works properly with json. @@ -84,13 +83,14 @@ private void runElasticSearchSinkTest(String index, Function + + org.testcontainers + elasticsearch + 1.15.0 + test + + org.apache.flink flink-test-utils_${scala.binary.version} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java deleted file mode 100644 index 8dc62168..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase; - -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.node.InternalSettingsPreparer; -import org.elasticsearch.node.Node; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.transport.Netty4Plugin; - -import java.io.File; -import java.util.Collections; - -/** - * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 6. - * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. - */ -public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { - - private Node node; - - @Override - public void start(File tmpDataFolder, String clusterName) throws Exception { - if (node == null) { - Settings settings = Settings.builder() - .put("cluster.name", clusterName) - .put("http.enabled", true) - .put("path.home", tmpDataFolder.getParent()) - .put("path.data", tmpDataFolder.getAbsolutePath()) - .build(); - - node = new PluginNode(settings); - node.start(); - } - } - - @Override - public void close() throws Exception { - if (node != null && !node.isClosed()) { - node.close(); - node = null; - } - } - - @Override - public Client getClient() { - if (node != null && !node.isClosed()) { - return node.client(); - } else { - return null; - } - } - - private static class PluginNode extends Node { - public PluginNode(Settings settings) { - super(InternalSettingsPreparer.prepareEnvironment(settings, null), Collections.>singletonList(Netty4Plugin.class)); - } - } - -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index ce57fadb..f1604d0d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -38,9 +37,14 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; import java.time.Duration; import java.time.LocalDate; @@ -61,7 +65,20 @@ public class Elasticsearch6DynamicSinkITCase { @ClassRule - public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource("es-6-dynamic-sink-tests"); + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName + .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("6.3.1")); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings) + .addTransportAddress(transportAddress); + } @Test public void testWritingDocuments() throws Exception { @@ -93,7 +110,7 @@ public void testWritingDocuments() throws Exception { .withSchema(schema) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") .build() ).getSinkRuntimeProvider(new MockContext()); @@ -104,7 +121,7 @@ public void testWritingDocuments() throws Exception { environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); @@ -141,7 +158,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); @@ -156,7 +173,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { LocalDateTime.parse("2012-12-12T12:12:12")) ).executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) .actionGet() .getSource(); @@ -193,7 +210,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); @@ -216,7 +233,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { LocalDateTime.parse("2013-12-12T13:13:13")) ).executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); // search API does not return documents that were not indexed, we might need to query // the index a few times @@ -279,14 +296,14 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) .executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 4a1d7e82..5663aab0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -24,13 +24,16 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; import org.apache.http.HttpHost; -import org.elasticsearch.client.RestClient; -import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.Client; import org.elasticsearch.client.RestHighLevelClient; -import org.junit.Before; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; -import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -39,13 +42,26 @@ */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - @Before - public void ensureClusterIsUp() throws IOException { - RestClientBuilder builder = RestClient.builder(HttpHost.create("http://127.0.0.1:9200")); - RestHighLevelClient client = new RestHighLevelClient(builder); - if (!client.ping()) { - throw new RuntimeException("Cannot ping cluster!"); - } + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName + .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("6.3.1")); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = getClusterName(); + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings) + .addTransportAddress(transportAddress); } @Test @@ -59,12 +75,12 @@ public void testElasticsearchSinkWithSmile() throws Exception { } @Test - public void testNullAddresses() throws Exception { + public void testNullAddresses() { runNullAddressesTest(); } @Test - public void testEmptyAddresses() throws Exception { + public void testEmptyAddresses() { runEmptyAddressesTest(); } @@ -90,10 +106,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( int bulkFlushMaxActions, String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + ElasticsearchSinkFunction> elasticsearchSinkFunction) { return createElasticsearchSinkForNode( - bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, elasticsearchContainer.getHttpHostAddress()); } @Override @@ -101,10 +117,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr int bulkFlushMaxActions, String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, - String ipAddress) throws Exception { + String hostAddress) { ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost(ipAddress, 9200, "http")); + httpHosts.add(HttpHost.create(hostAddress)); ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); builder.setBulkFlushMaxActions(bulkFlushMaxActions); diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index f1b9e822..e4393215 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -83,6 +83,13 @@ under the License. + + org.testcontainers + elasticsearch + 1.15.0 + test + + org.apache.flink flink-test-utils_${scala.binary.version} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java deleted file mode 100644 index 6f051a38..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase; - -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.node.InternalSettingsPreparer; -import org.elasticsearch.node.Node; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.transport.Netty4Plugin; - -import java.io.File; -import java.util.Collections; - -/** - * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 7. - * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. - */ -public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { - - private Node node; - - @Override - public void start(File tmpDataFolder, String clusterName) throws Exception { - if (node == null) { - Settings settings = Settings.builder() - .put("cluster.name", clusterName) - .put("http.cors.enabled", true) - .put("path.home", tmpDataFolder.getParent()) - .put("path.data", tmpDataFolder.getAbsolutePath()) - .build(); - - node = new PluginNode(settings); - node.start(); - } - } - - @Override - public void close() throws Exception { - if (node != null && !node.isClosed()) { - node.close(); - node = null; - } - } - - @Override - public Client getClient() { - if (node != null && !node.isClosed()) { - return node.client(); - } else { - return null; - } - } - - private static class PluginNode extends Node { - public PluginNode(Settings settings) { - super(InternalSettingsPreparer.prepareEnvironment(settings, Collections.emptyMap(), null, () -> "node1"), Collections.>singletonList(Netty4Plugin.class), true); - } - } - -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index d2964f55..c8bb4e8c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.testutils.ElasticsearchResource; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -38,9 +37,14 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; import java.time.Duration; import java.time.LocalDate; @@ -61,7 +65,20 @@ public class Elasticsearch7DynamicSinkITCase { @ClassRule - public static ElasticsearchResource elasticsearchResource = new ElasticsearchResource("es-dynamic-sink-it-test"); + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName + .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("7.5.1")); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings) + .addTransportAddress(transportAddress); + } @Test public void testWritingDocuments() throws Exception { @@ -91,7 +108,7 @@ public void testWritingDocuments() throws Exception { context() .withSchema(schema) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") .build() ).getSinkRuntimeProvider(new MockContext()); @@ -102,7 +119,7 @@ public void testWritingDocuments() throws Exception { environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); @@ -137,7 +154,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); @@ -152,7 +169,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { LocalDateTime.parse("2012-12-12T12:12:12")) ).executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); @@ -185,7 +202,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); @@ -208,7 +225,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { LocalDateTime.parse("2013-12-12T13:13:13")) ).executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); // search API does not return documents that were not indexed, we might need to query // the index a few times @@ -269,14 +286,14 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), "http://127.0.0.1:9200") + + String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ")"); tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) .executeInsert("esTable").await(); - Client client = elasticsearchResource.getClient(); + Client client = getClient(); Map response = client.get(new GetRequest("dynamic-index-2012-12-12", "1")) .actionGet() .getSource(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 66f1eabc..3ba3a8b8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -24,8 +24,15 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; import org.apache.http.HttpHost; +import org.elasticsearch.client.Client; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; import java.util.ArrayList; import java.util.List; @@ -35,6 +42,28 @@ */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName + .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("7.5.1")); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = getClusterName(); + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings) + .addTransportAddress(transportAddress); + } + @Test public void testElasticsearchSink() throws Exception { runElasticsearchSinkTest(); @@ -46,12 +75,12 @@ public void testElasticsearchSinkWithSmile() throws Exception { } @Test - public void testNullAddresses() throws Exception { + public void testNullAddresses() { runNullAddressesTest(); } @Test - public void testEmptyAddresses() throws Exception { + public void testEmptyAddresses() { runEmptyAddressesTest(); } @@ -77,10 +106,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( int bulkFlushMaxActions, String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + ElasticsearchSinkFunction> elasticsearchSinkFunction) { return createElasticsearchSinkForNode( - bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, elasticsearchContainer.getHttpHostAddress()); } @Override @@ -88,10 +117,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr int bulkFlushMaxActions, String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, - String ipAddress) throws Exception { + String hostAddress) { ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost(ipAddress, 9200, "http")); + httpHosts.add(HttpHost.create(hostAddress)); ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); builder.setBulkFlushMaxActions(bulkFlushMaxActions); From 45a699081e725814a793dafaebd1f957b017dfd1 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 26 Nov 2020 14:20:51 +0800 Subject: [PATCH 120/207] Update version to 1.13-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index fc37c05c..efed549b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index f6a53477..875a3c9d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index e4393215..bf213f62 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index efd1d306..80598c58 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 0ad181bf..2c0d3d9c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.12-SNAPSHOT + 1.13-SNAPSHOT .. From 355895636ccb2cae00ac66aacbaa3c5b1e91ddee Mon Sep 17 00:00:00 2001 From: zhisheng <1041218129@qq.com> Date: Wed, 9 Dec 2020 13:44:07 +0800 Subject: [PATCH 121/207] [hotfix][elasticsearch] Fix javadoc of Elasticsearch6Configuration This closes #14269 --- .../elasticsearch/table/Elasticsearch6Configuration.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java index c06898e8..de14831e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -30,7 +30,7 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; /** - * Elasticsearch 7 specific configuration. + * Elasticsearch 6 specific configuration. */ @Internal final class Elasticsearch6Configuration extends ElasticsearchConfiguration { From 80df7045c5a154f79bb0ed218cf5707df29b7ab1 Mon Sep 17 00:00:00 2001 From: Rufus Refactor Date: Mon, 28 Dec 2020 14:30:59 +0100 Subject: [PATCH 122/207] [FLINK-20651] Format code with Spotless/google-java-format --- .../ActionRequestFailureHandler.java | 76 +- .../BufferingNoOpRequestIndexer.java | 56 +- .../ElasticsearchApiCallBridge.java | 134 +- .../elasticsearch/ElasticsearchSinkBase.java | 852 ++++++------ .../ElasticsearchSinkFunction.java | 63 +- .../ElasticsearchUpsertTableSinkBase.java | 954 +++++++------ ...asticsearchUpsertTableSinkFactoryBase.java | 496 ++++--- ...PreElasticsearch6BulkProcessorIndexer.java | 83 +- .../elasticsearch/RequestIndexer.java | 85 +- .../index/AbstractTimeIndexGenerator.java | 24 +- .../elasticsearch/index/IndexGenerator.java | 19 +- .../index/IndexGeneratorBase.java | 44 +- .../index/IndexGeneratorFactory.java | 423 +++--- .../index/StaticIndexGenerator.java | 16 +- .../table/AbstractTimeIndexGenerator.java | 24 +- .../table/ElasticsearchConfiguration.java | 245 ++-- .../table/ElasticsearchOptions.java | 219 +-- .../table/ElasticsearchValidationUtils.java | 104 +- .../elasticsearch/table/IndexGenerator.java | 19 +- .../table/IndexGeneratorBase.java | 50 +- .../table/IndexGeneratorFactory.java | 403 +++--- .../elasticsearch/table/KeyExtractor.java | 159 ++- .../elasticsearch/table/RequestFactory.java | 49 +- .../table/RowElasticsearchSinkFunction.java | 183 ++- .../table/StaticIndexGenerator.java | 16 +- .../util/ElasticsearchUtils.java | 41 +- .../util/IgnoringFailureHandler.java | 15 +- .../util/NoOpFailureHandler.java | 38 +- .../RetryRejectedExecutionFailureHandler.java | 36 +- .../table/descriptors/Elasticsearch.java | 487 +++---- .../descriptors/ElasticsearchValidator.java | 304 ++-- .../ElasticsearchSinkBaseTest.java | 1238 +++++++++-------- .../ElasticsearchSinkTestBase.java | 319 ++--- ...csearchUpsertTableSinkFactoryTestBase.java | 364 ++--- .../EmbeddedElasticsearchNodeEnvironment.java | 47 +- .../index/IndexGeneratorTest.java | 420 +++--- .../table/IndexGeneratorFactoryTest.java | 338 ++--- .../elasticsearch/table/KeyExtractorTest.java | 183 +-- .../elasticsearch/table/TestContext.java | 82 +- .../testutils/ElasticsearchResource.java | 103 +- .../testutils/SourceSinkDataTestKit.java | 206 +-- .../table/descriptors/ElasticsearchTest.java | 249 ++-- .../table/Elasticsearch6Configuration.java | 85 +- .../table/Elasticsearch6DynamicSink.java | 521 +++---- .../Elasticsearch6DynamicSinkFactory.java | 202 ++- .../Elasticsearch6ApiCallBridge.java | 197 ++- .../Elasticsearch6BulkProcessorIndexer.java | 80 +- .../Elasticsearch6UpsertTableSink.java | 410 +++--- .../Elasticsearch6UpsertTableSinkFactory.java | 64 +- .../elasticsearch6/ElasticsearchSink.java | 375 ++--- .../elasticsearch6/RestClientFactory.java | 17 +- .../Elasticsearch6DynamicSinkFactoryTest.java | 385 +++-- .../Elasticsearch6DynamicSinkITCase.java | 543 ++++---- .../table/Elasticsearch6DynamicSinkTest.java | 433 +++--- ...sticsearch6UpsertTableSinkFactoryTest.java | 345 ++--- .../ElasticsearchSinkITCase.java | 186 +-- .../table/Elasticsearch7Configuration.java | 67 +- .../table/Elasticsearch7DynamicSink.java | 523 +++---- .../Elasticsearch7DynamicSinkFactory.java | 223 ++- .../Elasticsearch7ApiCallBridge.java | 200 +-- .../Elasticsearch7BulkProcessorIndexer.java | 80 +- .../Elasticsearch7UpsertTableSink.java | 441 +++--- .../Elasticsearch7UpsertTableSinkFactory.java | 62 +- .../elasticsearch7/ElasticsearchSink.java | 397 +++--- .../elasticsearch7/RestClientFactory.java | 17 +- .../Elasticsearch7DynamicSinkFactoryTest.java | 369 +++-- .../Elasticsearch7DynamicSinkITCase.java | 512 +++---- .../table/Elasticsearch7DynamicSinkTest.java | 433 +++--- ...sticsearch7UpsertTableSinkFactoryTest.java | 344 ++--- .../ElasticsearchSinkITCase.java | 186 +-- 70 files changed, 8692 insertions(+), 8271 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index 1514aba6..a13d10a9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -24,54 +24,56 @@ import java.io.Serializable; /** - * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how failed - * {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing malformed documents, or - * simply requesting them to be sent to Elasticsearch again if the failure is only temporary. + * An implementation of {@link ActionRequestFailureHandler} is provided by the user to define how + * failed {@link ActionRequest ActionRequests} should be handled, e.g. dropping them, reprocessing + * malformed documents, or simply requesting them to be sent to Elasticsearch again if the failure + * is only temporary. * *

Example: * *

{@code
+ * private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
  *
- *	private static class ExampleActionRequestFailureHandler implements ActionRequestFailureHandler {
- *
- *		@Override
- *		void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
- *			if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) {
- *				// full queue; re-add document for indexing
- *				indexer.add(action);
- *			} else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) {
- *				// malformed document; simply drop request without failing sink
- *			} else {
- *				// for all other failures, fail the sink;
- *				// here the failure is simply rethrown, but users can also choose to throw custom exceptions
- *				throw failure;
- *			}
- *		}
- *	}
+ * 	@Override
+ * 	void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable {
+ * 		if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) {
+ * 			// full queue; re-add document for indexing
+ * 			indexer.add(action);
+ * 		} else if (ExceptionUtils.findThrowable(failure, ElasticsearchParseException.class).isPresent()) {
+ * 			// malformed document; simply drop request without failing sink
+ * 		} else {
+ * 			// for all other failures, fail the sink;
+ * 			// here the failure is simply rethrown, but users can also choose to throw custom exceptions
+ * 			throw failure;
+ * 		}
+ * 	}
+ * }
  *
  * }
* - *

The above example will let the sink re-add requests that failed due to queue capacity saturation and drop requests - * with malformed documents, without failing the sink. For all other failures, the sink will fail. + *

The above example will let the sink re-add requests that failed due to queue capacity + * saturation and drop requests with malformed documents, without failing the sink. For all other + * failures, the sink will fail. * - *

Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the exact type - * could not be retrieved through the older version Java client APIs (thus, the types will be general {@link Exception}s - * and only differ in the failure message). In this case, it is recommended to match on the provided REST status code. + *

Note: For Elasticsearch 1.x, it is not feasible to match the type of the failure because the + * exact type could not be retrieved through the older version Java client APIs (thus, the types + * will be general {@link Exception}s and only differ in the failure message). In this case, it is + * recommended to match on the provided REST status code. */ @PublicEvolving public interface ActionRequestFailureHandler extends Serializable { - /** - * Handle a failed {@link ActionRequest}. - * - * @param action the {@link ActionRequest} that failed due to the failure - * @param failure the cause of failure - * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved) - * @param indexer request indexer to re-add the failed action, if intended to do so - * - * @throws Throwable if the sink should fail on this failure, the implementation should rethrow - * the exception or a custom one - */ - void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable; - + /** + * Handle a failed {@link ActionRequest}. + * + * @param action the {@link ActionRequest} that failed due to the failure + * @param failure the cause of failure + * @param restStatusCode the REST status code of the failure (-1 if none can be retrieved) + * @param indexer request indexer to re-add the failed action, if intended to do so + * @throws Throwable if the sink should fail on this failure, the implementation should rethrow + * the exception or a custom one + */ + void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java index 07341da8..4f86ba73 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java @@ -38,38 +38,38 @@ @NotThreadSafe class BufferingNoOpRequestIndexer implements RequestIndexer { - private ConcurrentLinkedQueue bufferedRequests; + private ConcurrentLinkedQueue bufferedRequests; - BufferingNoOpRequestIndexer() { - this.bufferedRequests = new ConcurrentLinkedQueue(); - } + BufferingNoOpRequestIndexer() { + this.bufferedRequests = new ConcurrentLinkedQueue(); + } - @Override - public void add(DeleteRequest... deleteRequests) { - Collections.addAll(bufferedRequests, deleteRequests); - } + @Override + public void add(DeleteRequest... deleteRequests) { + Collections.addAll(bufferedRequests, deleteRequests); + } - @Override - public void add(IndexRequest... indexRequests) { - Collections.addAll(bufferedRequests, indexRequests); - } + @Override + public void add(IndexRequest... indexRequests) { + Collections.addAll(bufferedRequests, indexRequests); + } - @Override - public void add(UpdateRequest... updateRequests) { - Collections.addAll(bufferedRequests, updateRequests); - } + @Override + public void add(UpdateRequest... updateRequests) { + Collections.addAll(bufferedRequests, updateRequests); + } - void processBufferedRequests(RequestIndexer actualIndexer) { - for (ActionRequest request : bufferedRequests) { - if (request instanceof IndexRequest) { - actualIndexer.add((IndexRequest) request); - } else if (request instanceof DeleteRequest) { - actualIndexer.add((DeleteRequest) request); - } else if (request instanceof UpdateRequest) { - actualIndexer.add((UpdateRequest) request); - } - } + void processBufferedRequests(RequestIndexer actualIndexer) { + for (ActionRequest request : bufferedRequests) { + if (request instanceof IndexRequest) { + actualIndexer.add((IndexRequest) request); + } else if (request instanceof DeleteRequest) { + actualIndexer.add((DeleteRequest) request); + } else if (request instanceof UpdateRequest) { + actualIndexer.add((UpdateRequest) request); + } + } - bufferedRequests.clear(); - } + bufferedRequests.clear(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 5a1ecf13..54e8399b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -31,83 +31,85 @@ import java.util.concurrent.atomic.AtomicLong; /** - * An {@link ElasticsearchApiCallBridge} is used to bridge incompatible Elasticsearch Java API calls across different versions. - * This includes calls to create Elasticsearch clients, handle failed item responses, etc. Any incompatible Elasticsearch - * Java APIs should be bridged using this interface. + * An {@link ElasticsearchApiCallBridge} is used to bridge incompatible Elasticsearch Java API calls + * across different versions. This includes calls to create Elasticsearch clients, handle failed + * item responses, etc. Any incompatible Elasticsearch Java APIs should be bridged using this + * interface. * - *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node - * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold - * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. + *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since + * connecting via an embedded node is allowed, the call bridge will hold reference to the created + * embedded node. Each instance of the sink will hold exactly one instance of the call bridge, and + * state cleanup is performed when the sink is closed. * * @param The Elasticsearch client, that implements {@link AutoCloseable}. */ @Internal public interface ElasticsearchApiCallBridge extends Serializable { - /** - * Creates an Elasticsearch client implementing {@link AutoCloseable}. - * - * @param clientConfig The configuration to use when constructing the client. - * @return The created client. - */ - C createClient(Map clientConfig); + /** + * Creates an Elasticsearch client implementing {@link AutoCloseable}. + * + * @param clientConfig The configuration to use when constructing the client. + * @return The created client. + */ + C createClient(Map clientConfig); - /** - * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. - * - * @param client the Elasticsearch client. - * @param listener the bulk processor listener. - * @return the bulk processor builder. - */ - BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); + /** + * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. + * + * @param client the Elasticsearch client. + * @param listener the bulk processor listener. + * @return the bulk processor builder. + */ + BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); - /** - * Extracts the cause of failure of a bulk item action. - * - * @param bulkItemResponse the bulk item response to extract cause of failure - * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). - */ - @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + /** + * Extracts the cause of failure of a bulk item action. + * + * @param bulkItemResponse the bulk item response to extract cause of failure + * @return the extracted {@link Throwable} from the response ({@code null} is the response is + * successful). + */ + @Nullable + Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); - /** - * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. - * The builder will be later on used to instantiate the actual {@link BulkProcessor}. - * - * @param builder the {@link BulkProcessor.Builder} to configure. - * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). - */ - void configureBulkProcessorBackoff( - BulkProcessor.Builder builder, - @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); + /** + * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. The builder + * will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user + * disabled backoff retries). + */ + void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); - /** - * Verify the client connection by making a test request/ping to the Elasticsearch cluster. - * - *

Called by {@link ElasticsearchSinkBase#open(org.apache.flink.configuration.Configuration)} after creating the client. This makes sure the underlying - * client is closed if the connection is not successful and preventing thread leak. - * - * @param client the Elasticsearch client. - */ - void verifyClientConnection(C client) throws IOException; + /** + * Verify the client connection by making a test request/ping to the Elasticsearch cluster. + * + *

Called by {@link ElasticsearchSinkBase#open(org.apache.flink.configuration.Configuration)} + * after creating the client. This makes sure the underlying client is closed if the connection + * is not successful and preventing thread leak. + * + * @param client the Elasticsearch client. + */ + void verifyClientConnection(C client) throws IOException; - /** - * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary compatible. - */ - default RequestIndexer createBulkProcessorIndexer( - BulkProcessor bulkProcessor, - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new PreElasticsearch6BulkProcessorIndexer( - bulkProcessor, - flushOnCheckpoint, - numPendingRequestsRef); - } - - /** - * Perform any necessary state cleanup. - */ - default void cleanup() { - // nothing to cleanup by default - } + /** + * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary + * compatible. + */ + default RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new PreElasticsearch6BulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + /** Perform any necessary state cleanup. */ + default void cleanup() { + // nothing to cleanup by default + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index d19fba67..15c64624 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -52,416 +52,456 @@ /** * Base class for all Flink Elasticsearch Sinks. * - *

This class implements the common behaviour across Elasticsearch versions, such as - * the use of an internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before - * sending the requests to the cluster, as well as passing input records to the user provided - * {@link ElasticsearchSinkFunction} for processing. + *

This class implements the common behaviour across Elasticsearch versions, such as the use of + * an internal {@link BulkProcessor} to buffer multiple {@link ActionRequest}s before sending the + * requests to the cluster, as well as passing input records to the user provided {@link + * ElasticsearchSinkFunction} for processing. * - *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of - * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, - * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. + *

The version specific API calls for different Elasticsearch versions should be defined by a + * concrete implementation of a {@link ElasticsearchApiCallBridge}, which is provided to the + * constructor of this class. This call bridge is used, for example, to create a Elasticsearch + * {@link Client}, handle failed item responses, etc. * * @param Type of the elements handled by this sink * @param Type of the Elasticsearch client, which implements {@link AutoCloseable} */ @Internal -public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { - - private static final long serialVersionUID = -1007596293618451942L; - - // ------------------------------------------------------------------------ - // Internal bulk processor configuration - // ------------------------------------------------------------------------ - - public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; - public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; - public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; - public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable"; - public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type"; - public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; - public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; - - /** - * Used to control whether the retry delay should increase exponentially or remain constant. - */ - @PublicEvolving - public enum FlushBackoffType { - CONSTANT, - EXPONENTIAL - } - - /** - * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to resource constraints - * (i.e. the client's internal thread pool is full), the backoff policy decides how long the bulk processor will - * wait before the operation is retried internally. - * - *

This is a proxy for version specific backoff policies. - */ - public static class BulkFlushBackoffPolicy implements Serializable { - - private static final long serialVersionUID = -6022851996101826049L; - - // the default values follow the Elasticsearch default settings for BulkProcessor - private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL; - private int maxRetryCount = 8; - private long delayMillis = 50; - - public FlushBackoffType getBackoffType() { - return backoffType; - } - - public int getMaxRetryCount() { - return maxRetryCount; - } - - public long getDelayMillis() { - return delayMillis; - } - - public void setBackoffType(FlushBackoffType backoffType) { - this.backoffType = checkNotNull(backoffType); - } - - public void setMaxRetryCount(int maxRetryCount) { - checkArgument(maxRetryCount >= 0); - this.maxRetryCount = maxRetryCount; - } - - public void setDelayMillis(long delayMillis) { - checkArgument(delayMillis >= 0); - this.delayMillis = delayMillis; - } - } - - private final Integer bulkProcessorFlushMaxActions; - private final Integer bulkProcessorFlushMaxSizeMb; - private final Long bulkProcessorFlushIntervalMillis; - private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; - - // ------------------------------------------------------------------------ - // User-facing API and configuration - // ------------------------------------------------------------------------ - - /** - * The config map that contains configuration for the bulk flushing behaviours. - * - *

For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this config - * map would also contain Elasticsearch-shipped configuration, and therefore this config map - * would also be forwarded when creating the Elasticsearch client. - */ - private final Map userConfig; - - /** The function that is used to construct multiple {@link ActionRequest ActionRequests} from each incoming element. */ - private final ElasticsearchSinkFunction elasticsearchSinkFunction; - - /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ - private final ActionRequestFailureHandler failureHandler; - - /** If true, the producer will wait until all outstanding action requests have been sent to Elasticsearch. */ - private boolean flushOnCheckpoint = true; - - /** Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest ActionRequests}. */ - private transient RequestIndexer requestIndexer; - - /** Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed requests. */ - private transient BufferingNoOpRequestIndexer failureRequestIndexer; - - // ------------------------------------------------------------------------ - // Internals for the Flink Elasticsearch Sink - // ------------------------------------------------------------------------ - - /** Call bridge for different version-specific. */ - private final ElasticsearchApiCallBridge callBridge; - - /** - * Number of pending action requests not yet acknowledged by Elasticsearch. - * This value is maintained only if {@link ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}. - * - *

This is incremented whenever the user adds (or re-adds through the {@link ActionRequestFailureHandler}) requests - * to the {@link RequestIndexer}. It is decremented for each completed request of a bulk request, in - * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, BulkResponse)} and - * {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, Throwable)}. - */ - private AtomicLong numPendingRequests = new AtomicLong(0); - - /** Elasticsearch client created using the call bridge. */ - private transient C client; - - /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ - private transient BulkProcessor bulkProcessor; - - /** - * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown in callbacks and - * the user considered it should fail the sink via the - * {@link ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. - * - *

Errors will be checked and rethrown before processing each input element, and when the sink is closed. - */ - private final AtomicReference failureThrowable = new AtomicReference<>(); - - public ElasticsearchSinkBase( - ElasticsearchApiCallBridge callBridge, - Map userConfig, - ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler) { - - this.callBridge = checkNotNull(callBridge); - this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); - this.failureHandler = checkNotNull(failureHandler); - // we eagerly check if the user-provided sink function and failure handler is serializable; - // otherwise, if they aren't serializable, users will merely get a non-informative error message - // "ElasticsearchSinkBase is not serializable" - - checkArgument(InstantiationUtil.isSerializable(elasticsearchSinkFunction), - "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + - "The object probably contains or references non-serializable fields."); - - checkArgument(InstantiationUtil.isSerializable(failureHandler), - "The implementation of the provided ActionRequestFailureHandler is not serializable. " + - "The object probably contains or references non-serializable fields."); - - // extract and remove bulk processor related configuration from the user-provided config, - // so that the resulting user config only contains configuration related to the Elasticsearch client. - - checkNotNull(userConfig); - - // copy config so we can remove entries without side-effects - userConfig = new HashMap<>(userConfig); - - ParameterTool params = ParameterTool.fromMap(userConfig); - - if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { - bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); - } else { - bulkProcessorFlushMaxActions = null; - } - - if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { - bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); - } else { - bulkProcessorFlushMaxSizeMb = null; - } - - if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { - bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); - } else { - bulkProcessorFlushIntervalMillis = null; - } - - boolean bulkProcessorFlushBackoffEnable = params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE); - - if (bulkProcessorFlushBackoffEnable) { - this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy(); - - if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) { - bulkProcessorFlushBackoffPolicy.setBackoffType(FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE))); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE); - } - - if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) { - bulkProcessorFlushBackoffPolicy.setMaxRetryCount(params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES); - } - - if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) { - bulkProcessorFlushBackoffPolicy.setDelayMillis(params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)); - userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY); - } - - } else { - bulkProcessorFlushBackoffPolicy = null; - } - - this.userConfig = userConfig; - } - - /** - * Disable flushing on checkpoint. When disabled, the sink will not wait for all - * pending action requests to be acknowledged by Elasticsearch on checkpoints. - * - *

NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT - * provide any strong guarantees for at-least-once delivery of action requests. - */ - public void disableFlushOnCheckpoint() { - this.flushOnCheckpoint = false; - } - - @Override - public void open(Configuration parameters) throws Exception { - client = callBridge.createClient(userConfig); - callBridge.verifyClientConnection(client); - bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); - requestIndexer = callBridge.createBulkProcessorIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequests); - failureRequestIndexer = new BufferingNoOpRequestIndexer(); - elasticsearchSinkFunction.open(); - } - - @Override - public void invoke(T value, Context context) throws Exception { - checkAsyncErrorsAndRequests(); - elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // no initialization needed - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - checkAsyncErrorsAndRequests(); - - if (flushOnCheckpoint) { - while (numPendingRequests.get() != 0) { - bulkProcessor.flush(); - checkAsyncErrorsAndRequests(); - } - } - } - - @Override - public void close() throws Exception { - elasticsearchSinkFunction.close(); - if (bulkProcessor != null) { - bulkProcessor.close(); - bulkProcessor = null; - } - - if (client != null) { - client.close(); - client = null; - } - - callBridge.cleanup(); - - // make sure any errors from callbacks are rethrown - checkErrorAndRethrow(); - } - - /** - * Build the {@link BulkProcessor}. - * - *

Note: this is exposed for testing purposes. - */ - @VisibleForTesting - protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { - checkNotNull(listener); - - BulkProcessor.Builder bulkProcessorBuilder = callBridge.createBulkProcessorBuilder(client, listener); - - // This makes flush() blocking - bulkProcessorBuilder.setConcurrentRequests(0); - - if (bulkProcessorFlushMaxActions != null) { - bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions); - } - - if (bulkProcessorFlushMaxSizeMb != null) { - bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, ByteSizeUnit.MB)); - } - - if (bulkProcessorFlushIntervalMillis != null) { - bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); - } - - // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null - callBridge.configureBulkProcessorBackoff(bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); - - return bulkProcessorBuilder.build(); - } - - private void checkErrorAndRethrow() { - Throwable cause = failureThrowable.get(); - if (cause != null) { - throw new RuntimeException("An error occurred in ElasticsearchSink.", cause); - } - } - - private void checkAsyncErrorsAndRequests() { - checkErrorAndRethrow(); - failureRequestIndexer.processBufferedRequests(requestIndexer); - } - - private class BulkProcessorListener implements BulkProcessor.Listener { - - @Override - public void beforeBulk(long executionId, BulkRequest request) { } - - @Override - public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { - if (response.hasFailures()) { - BulkItemResponse itemResponse; - Throwable failure; - RestStatus restStatus; - DocWriteRequest actionRequest; - - try { - for (int i = 0; i < response.getItems().length; i++) { - itemResponse = response.getItems()[i]; - failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); - if (failure != null) { - restStatus = itemResponse.getFailure().getStatus(); - actionRequest = request.requests().get(i); - if (restStatus == null) { - if (actionRequest instanceof ActionRequest) { - failureHandler.onFailure((ActionRequest) actionRequest, failure, -1, failureRequestIndexer); - } else { - throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); - } - } else { - if (actionRequest instanceof ActionRequest) { - failureHandler.onFailure((ActionRequest) actionRequest, failure, restStatus.getStatus(), failureRequestIndexer); - } else { - throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); - } - } - } - } - } catch (Throwable t) { - // fail the sink and skip the rest of the items - // if the failure handler decides to throw an exception - failureThrowable.compareAndSet(null, t); - } - } - - if (flushOnCheckpoint) { - numPendingRequests.getAndAdd(-request.numberOfActions()); - } - } - - @Override - public void afterBulk(long executionId, BulkRequest request, Throwable failure) { - try { - for (DocWriteRequest writeRequest : request.requests()) { - if (writeRequest instanceof ActionRequest) { - failureHandler.onFailure((ActionRequest) writeRequest, failure, -1, failureRequestIndexer); - } else { - throw new UnsupportedOperationException("The sink currently only supports ActionRequests"); - } - } - } catch (Throwable t) { - // fail the sink and skip the rest of the items - // if the failure handler decides to throw an exception - failureThrowable.compareAndSet(null, t); - } - - if (flushOnCheckpoint) { - numPendingRequests.getAndAdd(-request.numberOfActions()); - } - } - } - - @VisibleForTesting - long getNumPendingRequests() { - if (flushOnCheckpoint) { - return numPendingRequests.get(); - } else { - throw new UnsupportedOperationException( - "The number of pending requests is not maintained when flushing on checkpoint is disabled."); - } - } +public abstract class ElasticsearchSinkBase extends RichSinkFunction + implements CheckpointedFunction { + + private static final long serialVersionUID = -1007596293618451942L; + + // ------------------------------------------------------------------------ + // Internal bulk processor configuration + // ------------------------------------------------------------------------ + + public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = "bulk.flush.max.actions"; + public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = "bulk.flush.max.size.mb"; + public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = "bulk.flush.interval.ms"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = "bulk.flush.backoff.enable"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = "bulk.flush.backoff.type"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = "bulk.flush.backoff.retries"; + public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = "bulk.flush.backoff.delay"; + + /** Used to control whether the retry delay should increase exponentially or remain constant. */ + @PublicEvolving + public enum FlushBackoffType { + CONSTANT, + EXPONENTIAL + } + + /** + * Provides a backoff policy for bulk requests. Whenever a bulk request is rejected due to + * resource constraints (i.e. the client's internal thread pool is full), the backoff policy + * decides how long the bulk processor will wait before the operation is retried internally. + * + *

This is a proxy for version specific backoff policies. + */ + public static class BulkFlushBackoffPolicy implements Serializable { + + private static final long serialVersionUID = -6022851996101826049L; + + // the default values follow the Elasticsearch default settings for BulkProcessor + private FlushBackoffType backoffType = FlushBackoffType.EXPONENTIAL; + private int maxRetryCount = 8; + private long delayMillis = 50; + + public FlushBackoffType getBackoffType() { + return backoffType; + } + + public int getMaxRetryCount() { + return maxRetryCount; + } + + public long getDelayMillis() { + return delayMillis; + } + + public void setBackoffType(FlushBackoffType backoffType) { + this.backoffType = checkNotNull(backoffType); + } + + public void setMaxRetryCount(int maxRetryCount) { + checkArgument(maxRetryCount >= 0); + this.maxRetryCount = maxRetryCount; + } + + public void setDelayMillis(long delayMillis) { + checkArgument(delayMillis >= 0); + this.delayMillis = delayMillis; + } + } + + private final Integer bulkProcessorFlushMaxActions; + private final Integer bulkProcessorFlushMaxSizeMb; + private final Long bulkProcessorFlushIntervalMillis; + private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; + + // ------------------------------------------------------------------------ + // User-facing API and configuration + // ------------------------------------------------------------------------ + + /** + * The config map that contains configuration for the bulk flushing behaviours. + * + *

For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this + * config map would also contain Elasticsearch-shipped configuration, and therefore this config + * map would also be forwarded when creating the Elasticsearch client. + */ + private final Map userConfig; + + /** + * The function that is used to construct multiple {@link ActionRequest ActionRequests} from + * each incoming element. + */ + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + /** User-provided handler for failed {@link ActionRequest ActionRequests}. */ + private final ActionRequestFailureHandler failureHandler; + + /** + * If true, the producer will wait until all outstanding action requests have been sent to + * Elasticsearch. + */ + private boolean flushOnCheckpoint = true; + + /** + * Provided to the user via the {@link ElasticsearchSinkFunction} to add {@link ActionRequest + * ActionRequests}. + */ + private transient RequestIndexer requestIndexer; + + /** + * Provided to the {@link ActionRequestFailureHandler} to allow users to re-index failed + * requests. + */ + private transient BufferingNoOpRequestIndexer failureRequestIndexer; + + // ------------------------------------------------------------------------ + // Internals for the Flink Elasticsearch Sink + // ------------------------------------------------------------------------ + + /** Call bridge for different version-specific. */ + private final ElasticsearchApiCallBridge callBridge; + + /** + * Number of pending action requests not yet acknowledged by Elasticsearch. This value is + * maintained only if {@link ElasticsearchSinkBase#flushOnCheckpoint} is {@code true}. + * + *

This is incremented whenever the user adds (or re-adds through the {@link + * ActionRequestFailureHandler}) requests to the {@link RequestIndexer}. It is decremented for + * each completed request of a bulk request, in {@link BulkProcessor.Listener#afterBulk(long, + * BulkRequest, BulkResponse)} and {@link BulkProcessor.Listener#afterBulk(long, BulkRequest, + * Throwable)}. + */ + private AtomicLong numPendingRequests = new AtomicLong(0); + + /** Elasticsearch client created using the call bridge. */ + private transient C client; + + /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ + private transient BulkProcessor bulkProcessor; + + /** + * This is set from inside the {@link BulkProcessor.Listener} if a {@link Throwable} was thrown + * in callbacks and the user considered it should fail the sink via the {@link + * ActionRequestFailureHandler#onFailure(ActionRequest, Throwable, int, RequestIndexer)} method. + * + *

Errors will be checked and rethrown before processing each input element, and when the + * sink is closed. + */ + private final AtomicReference failureThrowable = new AtomicReference<>(); + + public ElasticsearchSinkBase( + ElasticsearchApiCallBridge callBridge, + Map userConfig, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler) { + + this.callBridge = checkNotNull(callBridge); + this.elasticsearchSinkFunction = checkNotNull(elasticsearchSinkFunction); + this.failureHandler = checkNotNull(failureHandler); + // we eagerly check if the user-provided sink function and failure handler is serializable; + // otherwise, if they aren't serializable, users will merely get a non-informative error + // message + // "ElasticsearchSinkBase is not serializable" + + checkArgument( + InstantiationUtil.isSerializable(elasticsearchSinkFunction), + "The implementation of the provided ElasticsearchSinkFunction is not serializable. " + + "The object probably contains or references non-serializable fields."); + + checkArgument( + InstantiationUtil.isSerializable(failureHandler), + "The implementation of the provided ActionRequestFailureHandler is not serializable. " + + "The object probably contains or references non-serializable fields."); + + // extract and remove bulk processor related configuration from the user-provided config, + // so that the resulting user config only contains configuration related to the + // Elasticsearch client. + + checkNotNull(userConfig); + + // copy config so we can remove entries without side-effects + userConfig = new HashMap<>(userConfig); + + ParameterTool params = ParameterTool.fromMap(userConfig); + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) { + bulkProcessorFlushMaxActions = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS); + } else { + bulkProcessorFlushMaxActions = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) { + bulkProcessorFlushMaxSizeMb = params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB); + } else { + bulkProcessorFlushMaxSizeMb = null; + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { + bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + } else { + bulkProcessorFlushIntervalMillis = null; + } + + boolean bulkProcessorFlushBackoffEnable = + params.getBoolean(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, true); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE); + + if (bulkProcessorFlushBackoffEnable) { + this.bulkProcessorFlushBackoffPolicy = new BulkFlushBackoffPolicy(); + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE)) { + bulkProcessorFlushBackoffPolicy.setBackoffType( + FlushBackoffType.valueOf(params.get(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE))); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)) { + bulkProcessorFlushBackoffPolicy.setMaxRetryCount( + params.getInt(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES); + } + + if (params.has(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)) { + bulkProcessorFlushBackoffPolicy.setDelayMillis( + params.getLong(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY)); + userConfig.remove(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY); + } + + } else { + bulkProcessorFlushBackoffPolicy = null; + } + + this.userConfig = userConfig; + } + + /** + * Disable flushing on checkpoint. When disabled, the sink will not wait for all pending action + * requests to be acknowledged by Elasticsearch on checkpoints. + * + *

NOTE: If flushing on checkpoint is disabled, the Flink Elasticsearch Sink does NOT provide + * any strong guarantees for at-least-once delivery of action requests. + */ + public void disableFlushOnCheckpoint() { + this.flushOnCheckpoint = false; + } + + @Override + public void open(Configuration parameters) throws Exception { + client = callBridge.createClient(userConfig); + callBridge.verifyClientConnection(client); + bulkProcessor = buildBulkProcessor(new BulkProcessorListener()); + requestIndexer = + callBridge.createBulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequests); + failureRequestIndexer = new BufferingNoOpRequestIndexer(); + elasticsearchSinkFunction.open(); + } + + @Override + public void invoke(T value, Context context) throws Exception { + checkAsyncErrorsAndRequests(); + elasticsearchSinkFunction.process(value, getRuntimeContext(), requestIndexer); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // no initialization needed + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + checkAsyncErrorsAndRequests(); + + if (flushOnCheckpoint) { + while (numPendingRequests.get() != 0) { + bulkProcessor.flush(); + checkAsyncErrorsAndRequests(); + } + } + } + + @Override + public void close() throws Exception { + elasticsearchSinkFunction.close(); + if (bulkProcessor != null) { + bulkProcessor.close(); + bulkProcessor = null; + } + + if (client != null) { + client.close(); + client = null; + } + + callBridge.cleanup(); + + // make sure any errors from callbacks are rethrown + checkErrorAndRethrow(); + } + + /** + * Build the {@link BulkProcessor}. + * + *

Note: this is exposed for testing purposes. + */ + @VisibleForTesting + protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { + checkNotNull(listener); + + BulkProcessor.Builder bulkProcessorBuilder = + callBridge.createBulkProcessorBuilder(client, listener); + + // This makes flush() blocking + bulkProcessorBuilder.setConcurrentRequests(0); + + if (bulkProcessorFlushMaxActions != null) { + bulkProcessorBuilder.setBulkActions(bulkProcessorFlushMaxActions); + } + + if (bulkProcessorFlushMaxSizeMb != null) { + bulkProcessorBuilder.setBulkSize( + new ByteSizeValue(bulkProcessorFlushMaxSizeMb, ByteSizeUnit.MB)); + } + + if (bulkProcessorFlushIntervalMillis != null) { + bulkProcessorBuilder.setFlushInterval( + TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + } + + // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null + callBridge.configureBulkProcessorBackoff( + bulkProcessorBuilder, bulkProcessorFlushBackoffPolicy); + + return bulkProcessorBuilder.build(); + } + + private void checkErrorAndRethrow() { + Throwable cause = failureThrowable.get(); + if (cause != null) { + throw new RuntimeException("An error occurred in ElasticsearchSink.", cause); + } + } + + private void checkAsyncErrorsAndRequests() { + checkErrorAndRethrow(); + failureRequestIndexer.processBufferedRequests(requestIndexer); + } + + private class BulkProcessorListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) {} + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + if (response.hasFailures()) { + BulkItemResponse itemResponse; + Throwable failure; + RestStatus restStatus; + DocWriteRequest actionRequest; + + try { + for (int i = 0; i < response.getItems().length; i++) { + itemResponse = response.getItems()[i]; + failure = callBridge.extractFailureCauseFromBulkItemResponse(itemResponse); + if (failure != null) { + restStatus = itemResponse.getFailure().getStatus(); + actionRequest = request.requests().get(i); + if (restStatus == null) { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + -1, + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } else { + if (actionRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) actionRequest, + failure, + restStatus.getStatus(), + failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + try { + for (DocWriteRequest writeRequest : request.requests()) { + if (writeRequest instanceof ActionRequest) { + failureHandler.onFailure( + (ActionRequest) writeRequest, failure, -1, failureRequestIndexer); + } else { + throw new UnsupportedOperationException( + "The sink currently only supports ActionRequests"); + } + } + } catch (Throwable t) { + // fail the sink and skip the rest of the items + // if the failure handler decides to throw an exception + failureThrowable.compareAndSet(null, t); + } + + if (flushOnCheckpoint) { + numPendingRequests.getAndAdd(-request.numberOfActions()); + } + } + } + + @VisibleForTesting + long getNumPendingRequests() { + if (flushOnCheckpoint) { + return numPendingRequests.get(); + } else { + throw new UnsupportedOperationException( + "The number of pending requests is not maintained when flushing on checkpoint is disabled."); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index d023a448..dfbeec84 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -34,24 +34,24 @@ *

Example: * *

{@code
- *					private static class TestElasticSearchSinkFunction implements
- *						ElasticsearchSinkFunction> {
+ * 				private static class TestElasticSearchSinkFunction implements
+ * 					ElasticsearchSinkFunction> {
  *
- *					public IndexRequest createIndexRequest(Tuple2 element) {
- *						Map json = new HashMap<>();
- *						json.put("data", element.f1);
+ * 				public IndexRequest createIndexRequest(Tuple2 element) {
+ * 					Map json = new HashMap<>();
+ * 					json.put("data", element.f1);
  *
- *						return Requests.indexRequest()
- *							.index("my-index")
- *							.type("my-type")
- *							.id(element.f0.toString())
- *							.source(json);
- *						}
+ * 					return Requests.indexRequest()
+ * 						.index("my-index")
+ * 						.type("my-type")
+ * 						.id(element.f0.toString())
+ * 						.source(json);
+ * 					}
  *
- *				public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
- *					indexer.add(createIndexRequest(element));
- *				}
- *		}
+ * 			public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
+ * 				indexer.add(createIndexRequest(element));
+ * 			}
+ * 	}
  *
  * }
* @@ -60,23 +60,22 @@ @PublicEvolving public interface ElasticsearchSinkFunction extends Serializable, Function { - /** - * Initialization method for the function. It is called once before the actual working process methods. - */ - default void open() throws Exception {} + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} - /** - * Tear-down method for the function. It is called when the sink closes. - */ - default void close () throws Exception {} + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} - /** - * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. - * The produced requests should be added to the provided {@link RequestIndexer}. - * - * @param element incoming element to process - * @param ctx runtime context containing information about the sink instance - * @param indexer request indexer that {@code ActionRequest} should be added to - */ - void process(T element, RuntimeContext ctx, RequestIndexer indexer); + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionsRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param ctx runtime context containing information about the sink instance + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void process(T element, RuntimeContext ctx, RequestIndexer indexer); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 10ed5a40..7037bc7f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -51,494 +51,474 @@ import java.util.Map; import java.util.Objects; -/** - * A version-agnostic Elasticsearch {@link UpsertStreamTableSink}. - */ +/** A version-agnostic Elasticsearch {@link UpsertStreamTableSink}. */ @Internal public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTableSink { - /** Flag that indicates that only inserts are accepted. */ - private final boolean isAppendOnly; - - /** Schema of the table. */ - private final TableSchema schema; - - /** Version-agnostic hosts configuration. */ - private final List hosts; - - /** Default index for all requests. */ - private final String index; - - /** Default document type for all requests. */ - private final String docType; - - /** Delimiter for composite keys. */ - private final String keyDelimiter; - - /** String literal for null keys. */ - private final String keyNullLiteral; - - /** Serialization schema used for the document. */ - private final SerializationSchema serializationSchema; - - /** Content type describing the serialization schema. */ - private final XContentType contentType; - - /** Failure handler for failing {@link ActionRequest}s. */ - private final ActionRequestFailureHandler failureHandler; - - /** - * Map of optional configuration parameters for the Elasticsearch sink. The config is - * internal and can change at any time. - */ - private final Map sinkOptions; - - /** - * Version-agnostic creation of {@link ActionRequest}s. - */ - private final RequestFactory requestFactory; - - /** Key field indices determined by the query. */ - private int[] keyFieldIndices = new int[0]; - - public ElasticsearchUpsertTableSinkBase( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - this.isAppendOnly = isAppendOnly; - this.schema = TableSchemaUtils.checkOnlyPhysicalColumns(schema); - this.hosts = Preconditions.checkNotNull(hosts); - this.index = Preconditions.checkNotNull(index); - this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); - this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); - this.docType = Preconditions.checkNotNull(docType); - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.failureHandler = Preconditions.checkNotNull(failureHandler); - this.sinkOptions = Preconditions.checkNotNull(sinkOptions); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - } - - @Override - public void setKeyFields(String[] keyNames) { - if (keyNames == null) { - this.keyFieldIndices = new int[0]; - return; - } - - final String[] fieldNames = getFieldNames(); - final int[] keyFieldIndices = new int[keyNames.length]; - for (int i = 0; i < keyNames.length; i++) { - keyFieldIndices[i] = -1; - for (int j = 0; j < fieldNames.length; j++) { - if (keyNames[i].equals(fieldNames[j])) { - keyFieldIndices[i] = j; - break; - } - } - if (keyFieldIndices[i] == -1) { - throw new RuntimeException("Invalid key fields: " + Arrays.toString(keyNames)); - } - } - - validateKeyTypes(keyFieldIndices); - - this.keyFieldIndices = keyFieldIndices; - } - - @Override - public void setIsAppendOnly(Boolean isAppendOnly) { - if (this.isAppendOnly && !isAppendOnly) { - throw new ValidationException( - "The given query is not supported by this sink because the sink is configured to " + - "operate in append mode only. Thus, it only support insertions (no queries " + - "with updating results)."); - } - } - - @Override - public TypeInformation getRecordType() { - return schema.toRowType(); - } - - @Override - public DataStreamSink consumeDataStream(DataStream> dataStream) { - final ElasticsearchUpsertSinkFunction upsertFunction = - new ElasticsearchUpsertSinkFunction( - IndexGeneratorFactory.createIndexGenerator(index, schema), - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - requestFactory, - keyFieldIndices); - final SinkFunction> sinkFunction = createSinkFunction( - hosts, - failureHandler, - sinkOptions, - upsertFunction); - return dataStream.addSink(sinkFunction) - .setParallelism(dataStream.getParallelism()) - .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TypeInformation> getOutputType() { - return Types.TUPLE(Types.BOOLEAN, getRecordType()); - } - - @Override - public String[] getFieldNames() { - return schema.getFieldNames(); - } - - @Override - public TypeInformation[] getFieldTypes() { - return schema.getFieldTypes(); - } - - @Override - public TableSink> configure(String[] fieldNames, TypeInformation[] fieldTypes) { - if (!Arrays.equals(getFieldNames(), fieldNames) || !Arrays.equals(getFieldTypes(), fieldTypes)) { - throw new ValidationException("Reconfiguration with different fields is not allowed. " + - "Expected: " + Arrays.toString(getFieldNames()) + " / " + Arrays.toString(getFieldTypes()) + ". " + - "But was: " + Arrays.toString(fieldNames) + " / " + Arrays.toString(fieldTypes)); - } - return copy( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - requestFactory); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ElasticsearchUpsertTableSinkBase that = (ElasticsearchUpsertTableSinkBase) o; - return Objects.equals(isAppendOnly, that.isAppendOnly) && - Objects.equals(schema, that.schema) && - Objects.equals(hosts, that.hosts) && - Objects.equals(index, that.index) && - Objects.equals(docType, that.docType) && - Objects.equals(keyDelimiter, that.keyDelimiter) && - Objects.equals(keyNullLiteral, that.keyNullLiteral) && - Objects.equals(serializationSchema, that.serializationSchema) && - Objects.equals(contentType, that.contentType) && - Objects.equals(failureHandler, that.failureHandler) && - Objects.equals(sinkOptions, that.sinkOptions); - } - - @Override - public int hashCode() { - return Objects.hash( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // For version-specific implementations - // -------------------------------------------------------------------------------------------- - - protected abstract ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory); - - protected abstract SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertFunction); - - // -------------------------------------------------------------------------------------------- - // Helper methods - // -------------------------------------------------------------------------------------------- - - /** - * Validate the types that are used for conversion to string. - */ - private void validateKeyTypes(int[] keyFieldIndices) { - final TypeInformation[] types = getFieldTypes(); - for (int keyFieldIndex : keyFieldIndices) { - final TypeInformation type = types[keyFieldIndex]; - if (!TypeCheckUtils.isSimpleStringRepresentation(type)) { - throw new ValidationException( - "Only simple types that can be safely converted into a string representation " + - "can be used as keys. But was: " + type); - } - } - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** - * Keys for optional parameterization of the sink. - */ - public enum SinkOption { - DISABLE_FLUSH_ON_CHECKPOINT, - BULK_FLUSH_MAX_ACTIONS, - BULK_FLUSH_MAX_SIZE, - BULK_FLUSH_INTERVAL, - BULK_FLUSH_BACKOFF_ENABLED, - BULK_FLUSH_BACKOFF_TYPE, - BULK_FLUSH_BACKOFF_RETRIES, - BULK_FLUSH_BACKOFF_DELAY, - REST_MAX_RETRY_TIMEOUT, - REST_PATH_PREFIX - } - - /** - * Entity for describing a host of Elasticsearch. - */ - public static class Host { - public final String hostname; - public final int port; - public final String protocol; - - public Host(String hostname, int port, String protocol) { - this.hostname = hostname; - this.port = port; - this.protocol = protocol; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Host host = (Host) o; - return port == host.port && - Objects.equals(hostname, host.hostname) && - Objects.equals(protocol, host.protocol); - } - - @Override - public int hashCode() { - return Objects.hash( - hostname, - port, - protocol); - } - - @Override - public String toString() { - return protocol + "://" - + hostname + ":" - + port; - } - } - - /** - * For version-agnostic creating of {@link ActionRequest}s. - */ - public interface RequestFactory extends Serializable { - - /** - * Creates an update request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document); - - /** - * Creates an index request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - IndexRequest createIndexRequest( - String index, - String docType, - XContentType contentType, - byte[] document); - - /** - * Creates a delete request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - DeleteRequest createDeleteRequest( - String index, - String docType, - String key); - } - - /** - * Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. - */ - public static class ElasticsearchUpsertSinkFunction implements ElasticsearchSinkFunction> { - - private final IndexGenerator indexGenerator; - private final String docType; - private final String keyDelimiter; - private final String keyNullLiteral; - private final SerializationSchema serializationSchema; - private final XContentType contentType; - private final RequestFactory requestFactory; - private final int[] keyFieldIndices; - - public ElasticsearchUpsertSinkFunction( - IndexGenerator indexGenerator, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - RequestFactory requestFactory, - int[] keyFieldIndices) { - - this.indexGenerator = Preconditions.checkNotNull(indexGenerator); - this.docType = Preconditions.checkNotNull(docType); - this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.keyFieldIndices = Preconditions.checkNotNull(keyFieldIndices); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); - } - - @Override - public void open() { - indexGenerator.open(); - } - - @Override - public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { - - final String formattedIndex = indexGenerator.generate(element.f1); - if (element.f0) { - processUpsert(element.f1, indexer, formattedIndex); - } else { - processDelete(element.f1, indexer, formattedIndex); - } - } - - private void processUpsert(Row row, RequestIndexer indexer, String formattedIndex) { - final byte[] document = serializationSchema.serialize(row); - if (keyFieldIndices.length == 0) { - final IndexRequest indexRequest = requestFactory.createIndexRequest( - formattedIndex, - docType, - contentType, - document); - indexer.add(indexRequest); - } else { - final String key = createKey(row); - final UpdateRequest updateRequest = requestFactory.createUpdateRequest( - formattedIndex, - docType, - key, - contentType, - document); - indexer.add(updateRequest); - } - } - - private void processDelete(Row row, RequestIndexer indexer, String formattedIndex) { - final String key = createKey(row); - final DeleteRequest deleteRequest = requestFactory.createDeleteRequest( - formattedIndex, - docType, - key); - indexer.add(deleteRequest); - } - - private String createKey(Row row) { - final StringBuilder builder = new StringBuilder(); - for (int i = 0; i < keyFieldIndices.length; i++) { - final int keyFieldIndex = keyFieldIndices[i]; - if (i > 0) { - builder.append(keyDelimiter); - } - final Object value = row.getField(keyFieldIndex); - if (value == null) { - builder.append(keyNullLiteral); - } else { - builder.append(value.toString()); - } - } - return builder.toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ElasticsearchUpsertSinkFunction that = (ElasticsearchUpsertSinkFunction) o; - return Objects.equals(indexGenerator, that.indexGenerator) && - Objects.equals(docType, that.docType) && - Objects.equals(keyDelimiter, that.keyDelimiter) && - Objects.equals(keyNullLiteral, that.keyNullLiteral) && - Objects.equals(serializationSchema, that.serializationSchema) && - contentType == that.contentType && - Objects.equals(requestFactory, that.requestFactory) && - Arrays.equals(keyFieldIndices, that.keyFieldIndices); - } - - @Override - public int hashCode() { - int result = Objects.hash( - indexGenerator, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - requestFactory); - result = 31 * result + Arrays.hashCode(keyFieldIndices); - return result; - } - } + /** Flag that indicates that only inserts are accepted. */ + private final boolean isAppendOnly; + + /** Schema of the table. */ + private final TableSchema schema; + + /** Version-agnostic hosts configuration. */ + private final List hosts; + + /** Default index for all requests. */ + private final String index; + + /** Default document type for all requests. */ + private final String docType; + + /** Delimiter for composite keys. */ + private final String keyDelimiter; + + /** String literal for null keys. */ + private final String keyNullLiteral; + + /** Serialization schema used for the document. */ + private final SerializationSchema serializationSchema; + + /** Content type describing the serialization schema. */ + private final XContentType contentType; + + /** Failure handler for failing {@link ActionRequest}s. */ + private final ActionRequestFailureHandler failureHandler; + + /** + * Map of optional configuration parameters for the Elasticsearch sink. The config is internal + * and can change at any time. + */ + private final Map sinkOptions; + + /** Version-agnostic creation of {@link ActionRequest}s. */ + private final RequestFactory requestFactory; + + /** Key field indices determined by the query. */ + private int[] keyFieldIndices = new int[0]; + + public ElasticsearchUpsertTableSinkBase( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + this.isAppendOnly = isAppendOnly; + this.schema = TableSchemaUtils.checkOnlyPhysicalColumns(schema); + this.hosts = Preconditions.checkNotNull(hosts); + this.index = Preconditions.checkNotNull(index); + this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); + this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); + this.docType = Preconditions.checkNotNull(docType); + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.failureHandler = Preconditions.checkNotNull(failureHandler); + this.sinkOptions = Preconditions.checkNotNull(sinkOptions); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + } + + @Override + public void setKeyFields(String[] keyNames) { + if (keyNames == null) { + this.keyFieldIndices = new int[0]; + return; + } + + final String[] fieldNames = getFieldNames(); + final int[] keyFieldIndices = new int[keyNames.length]; + for (int i = 0; i < keyNames.length; i++) { + keyFieldIndices[i] = -1; + for (int j = 0; j < fieldNames.length; j++) { + if (keyNames[i].equals(fieldNames[j])) { + keyFieldIndices[i] = j; + break; + } + } + if (keyFieldIndices[i] == -1) { + throw new RuntimeException("Invalid key fields: " + Arrays.toString(keyNames)); + } + } + + validateKeyTypes(keyFieldIndices); + + this.keyFieldIndices = keyFieldIndices; + } + + @Override + public void setIsAppendOnly(Boolean isAppendOnly) { + if (this.isAppendOnly && !isAppendOnly) { + throw new ValidationException( + "The given query is not supported by this sink because the sink is configured to " + + "operate in append mode only. Thus, it only support insertions (no queries " + + "with updating results)."); + } + } + + @Override + public TypeInformation getRecordType() { + return schema.toRowType(); + } + + @Override + public DataStreamSink consumeDataStream(DataStream> dataStream) { + final ElasticsearchUpsertSinkFunction upsertFunction = + new ElasticsearchUpsertSinkFunction( + IndexGeneratorFactory.createIndexGenerator(index, schema), + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + requestFactory, + keyFieldIndices); + final SinkFunction> sinkFunction = + createSinkFunction(hosts, failureHandler, sinkOptions, upsertFunction); + return dataStream + .addSink(sinkFunction) + .setParallelism(dataStream.getParallelism()) + .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); + } + + @Override + public TypeInformation> getOutputType() { + return Types.TUPLE(Types.BOOLEAN, getRecordType()); + } + + @Override + public String[] getFieldNames() { + return schema.getFieldNames(); + } + + @Override + public TypeInformation[] getFieldTypes() { + return schema.getFieldTypes(); + } + + @Override + public TableSink> configure( + String[] fieldNames, TypeInformation[] fieldTypes) { + if (!Arrays.equals(getFieldNames(), fieldNames) + || !Arrays.equals(getFieldTypes(), fieldTypes)) { + throw new ValidationException( + "Reconfiguration with different fields is not allowed. " + + "Expected: " + + Arrays.toString(getFieldNames()) + + " / " + + Arrays.toString(getFieldTypes()) + + ". " + + "But was: " + + Arrays.toString(fieldNames) + + " / " + + Arrays.toString(fieldTypes)); + } + return copy( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + requestFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchUpsertTableSinkBase that = (ElasticsearchUpsertTableSinkBase) o; + return Objects.equals(isAppendOnly, that.isAppendOnly) + && Objects.equals(schema, that.schema) + && Objects.equals(hosts, that.hosts) + && Objects.equals(index, that.index) + && Objects.equals(docType, that.docType) + && Objects.equals(keyDelimiter, that.keyDelimiter) + && Objects.equals(keyNullLiteral, that.keyNullLiteral) + && Objects.equals(serializationSchema, that.serializationSchema) + && Objects.equals(contentType, that.contentType) + && Objects.equals(failureHandler, that.failureHandler) + && Objects.equals(sinkOptions, that.sinkOptions); + } + + @Override + public int hashCode() { + return Objects.hash( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // For version-specific implementations + // -------------------------------------------------------------------------------------------- + + protected abstract ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory); + + protected abstract SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertFunction); + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + /** Validate the types that are used for conversion to string. */ + private void validateKeyTypes(int[] keyFieldIndices) { + final TypeInformation[] types = getFieldTypes(); + for (int keyFieldIndex : keyFieldIndices) { + final TypeInformation type = types[keyFieldIndex]; + if (!TypeCheckUtils.isSimpleStringRepresentation(type)) { + throw new ValidationException( + "Only simple types that can be safely converted into a string representation " + + "can be used as keys. But was: " + + type); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** Keys for optional parameterization of the sink. */ + public enum SinkOption { + DISABLE_FLUSH_ON_CHECKPOINT, + BULK_FLUSH_MAX_ACTIONS, + BULK_FLUSH_MAX_SIZE, + BULK_FLUSH_INTERVAL, + BULK_FLUSH_BACKOFF_ENABLED, + BULK_FLUSH_BACKOFF_TYPE, + BULK_FLUSH_BACKOFF_RETRIES, + BULK_FLUSH_BACKOFF_DELAY, + REST_MAX_RETRY_TIMEOUT, + REST_PATH_PREFIX + } + + /** Entity for describing a host of Elasticsearch. */ + public static class Host { + public final String hostname; + public final int port; + public final String protocol; + + public Host(String hostname, int port, String protocol) { + this.hostname = hostname; + this.port = port; + this.protocol = protocol; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Host host = (Host) o; + return port == host.port + && Objects.equals(hostname, host.hostname) + && Objects.equals(protocol, host.protocol); + } + + @Override + public int hashCode() { + return Objects.hash(hostname, port, protocol); + } + + @Override + public String toString() { + return protocol + "://" + hostname + ":" + port; + } + } + + /** For version-agnostic creating of {@link ActionRequest}s. */ + public interface RequestFactory extends Serializable { + + /** + * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field + * has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field + * has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, String docType, XContentType contentType, byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field + * has been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest(String index, String docType, String key); + } + + /** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ + public static class ElasticsearchUpsertSinkFunction + implements ElasticsearchSinkFunction> { + + private final IndexGenerator indexGenerator; + private final String docType; + private final String keyDelimiter; + private final String keyNullLiteral; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final int[] keyFieldIndices; + + public ElasticsearchUpsertSinkFunction( + IndexGenerator indexGenerator, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + int[] keyFieldIndices) { + + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = Preconditions.checkNotNull(docType); + this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.keyFieldIndices = Preconditions.checkNotNull(keyFieldIndices); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void process( + Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + + final String formattedIndex = indexGenerator.generate(element.f1); + if (element.f0) { + processUpsert(element.f1, indexer, formattedIndex); + } else { + processDelete(element.f1, indexer, formattedIndex); + } + } + + private void processUpsert(Row row, RequestIndexer indexer, String formattedIndex) { + final byte[] document = serializationSchema.serialize(row); + if (keyFieldIndices.length == 0) { + final IndexRequest indexRequest = + requestFactory.createIndexRequest( + formattedIndex, docType, contentType, document); + indexer.add(indexRequest); + } else { + final String key = createKey(row); + final UpdateRequest updateRequest = + requestFactory.createUpdateRequest( + formattedIndex, docType, key, contentType, document); + indexer.add(updateRequest); + } + } + + private void processDelete(Row row, RequestIndexer indexer, String formattedIndex) { + final String key = createKey(row); + final DeleteRequest deleteRequest = + requestFactory.createDeleteRequest(formattedIndex, docType, key); + indexer.add(deleteRequest); + } + + private String createKey(Row row) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < keyFieldIndices.length; i++) { + final int keyFieldIndex = keyFieldIndices[i]; + if (i > 0) { + builder.append(keyDelimiter); + } + final Object value = row.getField(keyFieldIndex); + if (value == null) { + builder.append(keyNullLiteral); + } else { + builder.append(value.toString()); + } + } + return builder.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchUpsertSinkFunction that = (ElasticsearchUpsertSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) + && Objects.equals(docType, that.docType) + && Objects.equals(keyDelimiter, that.keyDelimiter) + && Objects.equals(keyNullLiteral, that.keyNullLiteral) + && Objects.equals(serializationSchema, that.serializationSchema) + && contentType == that.contentType + && Objects.equals(requestFactory, that.requestFactory) + && Arrays.equals(keyFieldIndices, that.keyFieldIndices); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + indexGenerator, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + requestFactory); + result = 31 * result + Arrays.hashCode(keyFieldIndices); + return result; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index a20cb330..dec4b30f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -99,223 +99,281 @@ * Version-agnostic table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch. */ @Internal -public abstract class ElasticsearchUpsertTableSinkFactoryBase implements StreamTableSinkFactory> { - - private static final String SUPPORTED_FORMAT_TYPE = "json"; - private static final XContentType SUPPORTED_CONTENT_TYPE = XContentType.JSON; - private static final String DEFAULT_KEY_DELIMITER = "_"; - private static final String DEFAULT_KEY_NULL_LITERAL = "null"; - private static final String DEFAULT_FAILURE_HANDLER = CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; - - @Override - public Map requiredContext() { - final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH); - context.put(CONNECTOR_VERSION, elasticsearchVersion()); - context.put(CONNECTOR_PROPERTY_VERSION, "1"); - return context; - } - - @Override - public List supportedProperties() { - final List properties = new ArrayList<>(); - - // streaming properties - properties.add(UPDATE_MODE); - - // Elasticsearch - properties.add(CONNECTOR_HOSTS); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PORT); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PROTOCOL); - properties.add(CONNECTOR_INDEX); - properties.add(CONNECTOR_DOCUMENT_TYPE); - properties.add(CONNECTOR_KEY_DELIMITER); - properties.add(CONNECTOR_KEY_NULL_LITERAL); - properties.add(CONNECTOR_FAILURE_HANDLER); - properties.add(CONNECTOR_FAILURE_HANDLER_CLASS); - properties.add(CONNECTOR_FLUSH_ON_CHECKPOINT); - properties.add(CONNECTOR_BULK_FLUSH_MAX_ACTIONS); - properties.add(CONNECTOR_BULK_FLUSH_MAX_SIZE); - properties.add(CONNECTOR_BULK_FLUSH_INTERVAL); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY); - properties.add(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT); - properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); - - // schema - properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_NAME); - // computed column - properties.add(SCHEMA + ".#." + EXPR); - - // watermark - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); - - // table constraint - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); - - // format wildcard - properties.add(FORMAT + ".*"); - - return properties; - } - - @Override - public StreamTableSink> createStreamTableSink(Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - return createElasticsearchUpsertTableSink( - descriptorProperties.isValue(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND), - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)), - getHosts(descriptorProperties), - descriptorProperties.getString(CONNECTOR_INDEX), - descriptorProperties.getString(CONNECTOR_DOCUMENT_TYPE), - descriptorProperties.getOptionalString(CONNECTOR_KEY_DELIMITER).orElse(DEFAULT_KEY_DELIMITER), - descriptorProperties.getOptionalString(CONNECTOR_KEY_NULL_LITERAL).orElse(DEFAULT_KEY_NULL_LITERAL), - getSerializationSchema(properties), - SUPPORTED_CONTENT_TYPE, - getFailureHandler(descriptorProperties), - getSinkOptions(descriptorProperties)); - } - - // -------------------------------------------------------------------------------------------- - // For version-specific factories - // -------------------------------------------------------------------------------------------- - - protected abstract String elasticsearchVersion(); - - protected abstract ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions); - - // -------------------------------------------------------------------------------------------- - // Helper methods - // -------------------------------------------------------------------------------------------- - - private DescriptorProperties getValidatedProperties(Map properties) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putProperties(properties); - - new StreamTableDescriptorValidator(true, false, true).validate(descriptorProperties); - new SchemaValidator(true, false, false).validate(descriptorProperties); - new ElasticsearchValidator().validate(descriptorProperties); - - return descriptorProperties; - } - - private List getHosts(DescriptorProperties descriptorProperties) { - if (descriptorProperties.containsKey(CONNECTOR_HOSTS)) { - return validateAndParseHostsString(descriptorProperties); - } else { - final List> hosts = descriptorProperties.getFixedIndexedProperties( - CONNECTOR_HOSTS, - Arrays.asList(CONNECTOR_HOSTS_HOSTNAME, CONNECTOR_HOSTS_PORT, CONNECTOR_HOSTS_PROTOCOL)); - return hosts.stream() - .map(host -> new Host( - descriptorProperties.getString(host.get(CONNECTOR_HOSTS_HOSTNAME)), - descriptorProperties.getInt(host.get(CONNECTOR_HOSTS_PORT)), - descriptorProperties.getString(host.get(CONNECTOR_HOSTS_PROTOCOL)))) - .collect(Collectors.toList()); - } - } - - private SerializationSchema getSerializationSchema(Map properties) { - final String formatType = properties.get(FORMAT_TYPE); - // we could have added this check to the table factory context - // but this approach allows to throw more helpful error messages - // if the supported format has not been added - if (formatType == null || !formatType.equals(SUPPORTED_FORMAT_TYPE)) { - throw new ValidationException( - "The Elasticsearch sink requires a '" + SUPPORTED_FORMAT_TYPE + "' format."); - } - - @SuppressWarnings("unchecked") - final SerializationSchemaFactory formatFactory = TableFactoryService.find( - SerializationSchemaFactory.class, - properties, - this.getClass().getClassLoader()); - return formatFactory.createSerializationSchema(properties); - } - - private ActionRequestFailureHandler getFailureHandler(DescriptorProperties descriptorProperties) { - final String failureHandler = descriptorProperties - .getOptionalString(CONNECTOR_FAILURE_HANDLER) - .orElse(DEFAULT_FAILURE_HANDLER); - switch (failureHandler) { - case CONNECTOR_FAILURE_HANDLER_VALUE_FAIL: - return new NoOpFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE: - return new IgnoringFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_RETRY: - return new RetryRejectedExecutionFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM: - final Class clazz = descriptorProperties - .getClass(CONNECTOR_FAILURE_HANDLER_CLASS, ActionRequestFailureHandler.class); - return InstantiationUtil.instantiate(clazz); - default: - throw new IllegalArgumentException("Unknown failure handler."); - } - } - - private Map getSinkOptions(DescriptorProperties descriptorProperties) { - final Map options = new HashMap<>(); - - descriptorProperties.getOptionalBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT) - .ifPresent(v -> options.put(SinkOption.DISABLE_FLUSH_ON_CHECKPOINT, String.valueOf(!v))); - - mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_MAX_ACTIONS, SinkOption.BULK_FLUSH_MAX_ACTIONS); - mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_MAX_SIZE, SinkOption.BULK_FLUSH_MAX_SIZE); - mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_INTERVAL, SinkOption.BULK_FLUSH_INTERVAL); - - descriptorProperties.getOptionalString(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE) - .ifPresent(v -> { - options.put( - SinkOption.BULK_FLUSH_BACKOFF_ENABLED, - String.valueOf(!v.equals(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED))); - switch (v) { - case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT: - options.put( - SinkOption.BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchSinkBase.FlushBackoffType.CONSTANT.toString()); - break; - case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL: - options.put( - SinkOption.BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL.toString()); - break; - default: - throw new IllegalArgumentException("Unknown backoff type."); - } - }); - - mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, SinkOption.BULK_FLUSH_BACKOFF_RETRIES); - mapSinkOption(descriptorProperties, options, CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, SinkOption.BULK_FLUSH_BACKOFF_DELAY); - mapSinkOption(descriptorProperties, options, CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, SinkOption.REST_MAX_RETRY_TIMEOUT); - mapSinkOption(descriptorProperties, options, CONNECTOR_CONNECTION_PATH_PREFIX, SinkOption.REST_PATH_PREFIX); - - return options; - } - - private void mapSinkOption( - DescriptorProperties descriptorProperties, - Map options, - String fromKey, - SinkOption toKey) { - descriptorProperties.getOptionalString(fromKey).ifPresent(v -> options.put(toKey, v)); - } +public abstract class ElasticsearchUpsertTableSinkFactoryBase + implements StreamTableSinkFactory> { + + private static final String SUPPORTED_FORMAT_TYPE = "json"; + private static final XContentType SUPPORTED_CONTENT_TYPE = XContentType.JSON; + private static final String DEFAULT_KEY_DELIMITER = "_"; + private static final String DEFAULT_KEY_NULL_LITERAL = "null"; + private static final String DEFAULT_FAILURE_HANDLER = CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; + + @Override + public Map requiredContext() { + final Map context = new HashMap<>(); + context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH); + context.put(CONNECTOR_VERSION, elasticsearchVersion()); + context.put(CONNECTOR_PROPERTY_VERSION, "1"); + return context; + } + + @Override + public List supportedProperties() { + final List properties = new ArrayList<>(); + + // streaming properties + properties.add(UPDATE_MODE); + + // Elasticsearch + properties.add(CONNECTOR_HOSTS); + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PORT); + properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PROTOCOL); + properties.add(CONNECTOR_INDEX); + properties.add(CONNECTOR_DOCUMENT_TYPE); + properties.add(CONNECTOR_KEY_DELIMITER); + properties.add(CONNECTOR_KEY_NULL_LITERAL); + properties.add(CONNECTOR_FAILURE_HANDLER); + properties.add(CONNECTOR_FAILURE_HANDLER_CLASS); + properties.add(CONNECTOR_FLUSH_ON_CHECKPOINT); + properties.add(CONNECTOR_BULK_FLUSH_MAX_ACTIONS); + properties.add(CONNECTOR_BULK_FLUSH_MAX_SIZE); + properties.add(CONNECTOR_BULK_FLUSH_INTERVAL); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES); + properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY); + properties.add(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT); + properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); + + // schema + properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_TYPE); + properties.add(SCHEMA + ".#." + SCHEMA_NAME); + // computed column + properties.add(SCHEMA + ".#." + EXPR); + + // watermark + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); + properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); + + // table constraint + properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); + properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); + + // format wildcard + properties.add(FORMAT + ".*"); + + return properties; + } + + @Override + public StreamTableSink> createStreamTableSink( + Map properties) { + final DescriptorProperties descriptorProperties = getValidatedProperties(properties); + + return createElasticsearchUpsertTableSink( + descriptorProperties.isValue(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND), + TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)), + getHosts(descriptorProperties), + descriptorProperties.getString(CONNECTOR_INDEX), + descriptorProperties.getString(CONNECTOR_DOCUMENT_TYPE), + descriptorProperties + .getOptionalString(CONNECTOR_KEY_DELIMITER) + .orElse(DEFAULT_KEY_DELIMITER), + descriptorProperties + .getOptionalString(CONNECTOR_KEY_NULL_LITERAL) + .orElse(DEFAULT_KEY_NULL_LITERAL), + getSerializationSchema(properties), + SUPPORTED_CONTENT_TYPE, + getFailureHandler(descriptorProperties), + getSinkOptions(descriptorProperties)); + } + + // -------------------------------------------------------------------------------------------- + // For version-specific factories + // -------------------------------------------------------------------------------------------- + + protected abstract String elasticsearchVersion(); + + protected abstract ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions); + + // -------------------------------------------------------------------------------------------- + // Helper methods + // -------------------------------------------------------------------------------------------- + + private DescriptorProperties getValidatedProperties(Map properties) { + final DescriptorProperties descriptorProperties = new DescriptorProperties(true); + descriptorProperties.putProperties(properties); + + new StreamTableDescriptorValidator(true, false, true).validate(descriptorProperties); + new SchemaValidator(true, false, false).validate(descriptorProperties); + new ElasticsearchValidator().validate(descriptorProperties); + + return descriptorProperties; + } + + private List getHosts(DescriptorProperties descriptorProperties) { + if (descriptorProperties.containsKey(CONNECTOR_HOSTS)) { + return validateAndParseHostsString(descriptorProperties); + } else { + final List> hosts = + descriptorProperties.getFixedIndexedProperties( + CONNECTOR_HOSTS, + Arrays.asList( + CONNECTOR_HOSTS_HOSTNAME, + CONNECTOR_HOSTS_PORT, + CONNECTOR_HOSTS_PROTOCOL)); + return hosts.stream() + .map( + host -> + new Host( + descriptorProperties.getString( + host.get(CONNECTOR_HOSTS_HOSTNAME)), + descriptorProperties.getInt( + host.get(CONNECTOR_HOSTS_PORT)), + descriptorProperties.getString( + host.get(CONNECTOR_HOSTS_PROTOCOL)))) + .collect(Collectors.toList()); + } + } + + private SerializationSchema getSerializationSchema(Map properties) { + final String formatType = properties.get(FORMAT_TYPE); + // we could have added this check to the table factory context + // but this approach allows to throw more helpful error messages + // if the supported format has not been added + if (formatType == null || !formatType.equals(SUPPORTED_FORMAT_TYPE)) { + throw new ValidationException( + "The Elasticsearch sink requires a '" + SUPPORTED_FORMAT_TYPE + "' format."); + } + + @SuppressWarnings("unchecked") + final SerializationSchemaFactory formatFactory = + TableFactoryService.find( + SerializationSchemaFactory.class, + properties, + this.getClass().getClassLoader()); + return formatFactory.createSerializationSchema(properties); + } + + private ActionRequestFailureHandler getFailureHandler( + DescriptorProperties descriptorProperties) { + final String failureHandler = + descriptorProperties + .getOptionalString(CONNECTOR_FAILURE_HANDLER) + .orElse(DEFAULT_FAILURE_HANDLER); + switch (failureHandler) { + case CONNECTOR_FAILURE_HANDLER_VALUE_FAIL: + return new NoOpFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE: + return new IgnoringFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_RETRY: + return new RetryRejectedExecutionFailureHandler(); + case CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM: + final Class clazz = + descriptorProperties.getClass( + CONNECTOR_FAILURE_HANDLER_CLASS, ActionRequestFailureHandler.class); + return InstantiationUtil.instantiate(clazz); + default: + throw new IllegalArgumentException("Unknown failure handler."); + } + } + + private Map getSinkOptions(DescriptorProperties descriptorProperties) { + final Map options = new HashMap<>(); + + descriptorProperties + .getOptionalBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT) + .ifPresent( + v -> + options.put( + SinkOption.DISABLE_FLUSH_ON_CHECKPOINT, + String.valueOf(!v))); + + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_BULK_FLUSH_MAX_ACTIONS, + SinkOption.BULK_FLUSH_MAX_ACTIONS); + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_BULK_FLUSH_MAX_SIZE, + SinkOption.BULK_FLUSH_MAX_SIZE); + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_BULK_FLUSH_INTERVAL, + SinkOption.BULK_FLUSH_INTERVAL); + + descriptorProperties + .getOptionalString(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE) + .ifPresent( + v -> { + options.put( + SinkOption.BULK_FLUSH_BACKOFF_ENABLED, + String.valueOf( + !v.equals( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED))); + switch (v) { + case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT: + options.put( + SinkOption.BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchSinkBase.FlushBackoffType.CONSTANT + .toString()); + break; + case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL: + options.put( + SinkOption.BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL + .toString()); + break; + default: + throw new IllegalArgumentException("Unknown backoff type."); + } + }); + + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, + SinkOption.BULK_FLUSH_BACKOFF_RETRIES); + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, + SinkOption.BULK_FLUSH_BACKOFF_DELAY); + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, + SinkOption.REST_MAX_RETRY_TIMEOUT); + mapSinkOption( + descriptorProperties, + options, + CONNECTOR_CONNECTION_PATH_PREFIX, + SinkOption.REST_PATH_PREFIX); + + return options; + } + + private void mapSinkOption( + DescriptorProperties descriptorProperties, + Map options, + String fromKey, + SinkOption toKey) { + descriptorProperties.getOptionalString(fromKey).ifPresent(v -> options.put(toKey, v)); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java index 85f4b9a3..50801abf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java @@ -31,54 +31,57 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. - * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. * - * @deprecated This class is not binary compatible with newer Elasticsearch 6+ versions - * (i.e. the {@link #add(UpdateRequest...)} ). However, this module is currently - * compiled against a very old Elasticsearch version. + * @deprecated This class is not binary compatible with newer Elasticsearch 6+ versions (i.e. the + * {@link #add(UpdateRequest...)} ). However, this module is currently compiled against a very + * old Elasticsearch version. */ @Deprecated @Internal class PreElasticsearch6BulkProcessorIndexer implements RequestIndexer { - private final BulkProcessor bulkProcessor; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; - PreElasticsearch6BulkProcessorIndexer(BulkProcessor bulkProcessor, boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { - this.bulkProcessor = checkNotNull(bulkProcessor); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } + PreElasticsearch6BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(deleteRequest); - } - } + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(indexRequest); - } - } + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(updateRequest); - } - } + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index 3dc8f879..3cf748f6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -26,51 +26,56 @@ import org.elasticsearch.action.update.UpdateRequest; /** - * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare - * them for sending to an Elasticsearch cluster. + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Elasticsearch cluster. */ @PublicEvolving public interface RequestIndexer { - /** - * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to Elasticsearch. - * - * @param actionRequests The multiple {@link ActionRequest} to add. - * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} - */ - @Deprecated - default void add(ActionRequest... actionRequests) { - for (ActionRequest actionRequest : actionRequests) { - if (actionRequest instanceof IndexRequest) { - add((IndexRequest) actionRequest); - } else if (actionRequest instanceof DeleteRequest) { - add((DeleteRequest) actionRequest); - } else if (actionRequest instanceof UpdateRequest) { - add((UpdateRequest) actionRequest); - } else { - throw new IllegalArgumentException("RequestIndexer only supports Index, Delete and Update requests"); - } - } - } + /** + * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param actionRequests The multiple {@link ActionRequest} to add. + * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} + */ + @Deprecated + default void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + if (actionRequest instanceof IndexRequest) { + add((IndexRequest) actionRequest); + } else if (actionRequest instanceof DeleteRequest) { + add((DeleteRequest) actionRequest); + } else if (actionRequest instanceof UpdateRequest) { + add((UpdateRequest) actionRequest); + } else { + throw new IllegalArgumentException( + "RequestIndexer only supports Index, Delete and Update requests"); + } + } + } - /** - * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to Elasticsearch. - * - * @param deleteRequests The multiple {@link DeleteRequest} to add. - */ - void add(DeleteRequest... deleteRequests); + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); - /** - * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to Elasticsearch. - * - * @param indexRequests The multiple {@link IndexRequest} to add. - */ - void add(IndexRequest... indexRequests); + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); - /** - * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to Elasticsearch. - * - * @param updateRequests The multiple {@link UpdateRequest} to add. - */ - void add(UpdateRequest... updateRequests); + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java index 52141450..1625ef5c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java @@ -20,21 +20,19 @@ import java.time.format.DateTimeFormatter; -/** - * Abstract class for time related {@link IndexGenerator}. - */ +/** Abstract class for time related {@link IndexGenerator}. */ public abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { - private final String dateTimeFormat; - protected transient DateTimeFormatter dateTimeFormatter; + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; - public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { - super(index); - this.dateTimeFormat = dateTimeFormat; - } + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } - @Override - public void open() { - this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); - } + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java index d45fc1be..5f390f23 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java @@ -23,19 +23,16 @@ import java.io.Serializable; -/** - * This interface is responsible to generate index name from given {@link Row} record. - */ +/** This interface is responsible to generate index name from given {@link Row} record. */ @Internal public interface IndexGenerator extends Serializable { - /** - * Initialize the index generator, this will be called only once before {@link #generate(Row)} is called. - */ - default void open() {} + /** + * Initialize the index generator, this will be called only once before {@link #generate(Row)} + * is called. + */ + default void open() {} - /** - * Generate index name according the the given row. - */ - String generate(Row row); + /** Generate index name according the the given row. */ + String generate(Row row); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java index 00d95e9e..1e08fa0c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java @@ -20,32 +20,30 @@ import java.util.Objects; -/** - * Base class for {@link IndexGenerator}. - */ +/** Base class for {@link IndexGenerator}. */ public abstract class IndexGeneratorBase implements IndexGenerator { - private static final long serialVersionUID = 1L; - protected final String index; + private static final long serialVersionUID = 1L; + protected final String index; - public IndexGeneratorBase(String index) { - this.index = index; - } + public IndexGeneratorBase(String index) { + this.index = index; + } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof IndexGeneratorBase)) { - return false; - } - IndexGeneratorBase that = (IndexGeneratorBase) o; - return index.equals(that.index); - } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } - @Override - public int hashCode() { - return Objects.hash(index); - } + @Override + public int hashCode() { + return Objects.hash(index); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java index 9f84051d..bd84d9b3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java @@ -46,234 +46,237 @@ * *

Flink supports both static index and dynamic index. * - *

If you want to have a static index, this option value should be a plain string, e.g. 'myusers', - * all the records will be consistently written into "myusers" index. + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. * - *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in the - * record to dynamically generate a target index. You can also use '{field_name|date_format_string}' to - * convert a field value of TIMESTAMP/DATE/TIME type into the format specified by date_format_string. The - * date_format_string is compatible with {@link java.text.SimpleDateFormat}. For example, if the option - * value is 'myusers_{log_ts|yyyy-MM-dd}', then a record with log_ts field value 2020-03-27 12:25:55 will - * be written into "myusers-2020-03-27" index. + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers-2020-03-27" index. */ @Internal public class IndexGeneratorFactory { - private IndexGeneratorFactory() {} + private IndexGeneratorFactory() {} - public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { - final IndexHelper indexHelper = new IndexHelper(); - if (indexHelper.checkIsDynamicIndex(index)) { - return createRuntimeIndexGenerator(index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); - } else { - return new StaticIndexGenerator(index); - } - } + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } - private static IndexGenerator createRuntimeIndexGenerator( - String index, - String[] fieldNames, - DataType[] fieldTypes, - IndexHelper indexHelper) { - final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); - final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); - final String indexSuffix = index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + private static IndexGenerator createRuntimeIndexGenerator( + String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); - final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); - final int indexFieldPos = indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); - final TypeInformation indexFieldType = TypeConversions.fromDataTypeToLegacyInfo(fieldTypes[indexFieldPos]); + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final TypeInformation indexFieldType = + TypeConversions.fromDataTypeToLegacyInfo(fieldTypes[indexFieldPos]); - // validate index field type - indexHelper.validateIndexFieldType(indexFieldType); + // validate index field type + indexHelper.validateIndexFieldType(indexFieldType); - // time extract dynamic index pattern - if (isDynamicIndexWithFormat) { - final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldType); - // DataTypes.SQL_TIMESTAMP - if (indexFieldType == Types.LOCAL_DATE_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalDateTime indexField = (LocalDateTime) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } - else if (indexFieldType == Types.SQL_TIMESTAMP) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Timestamp indexField = (Timestamp) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.toLocalDateTime().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } - // DataTypes.SQL_DATE - else if (indexFieldType == Types.LOCAL_DATE) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalDate indexField = (LocalDate) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else if (indexFieldType == Types.SQL_DATE) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Date indexField = (Date) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.toLocalDate().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } // DataTypes.TIME - else if (indexFieldType == Types.LOCAL_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalTime indexField = (LocalTime) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else if (indexFieldType == Types.SQL_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Time indexField = (Time) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.toLocalTime().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else { - throw new TableException(String.format("Unsupported type '%s' found in Elasticsearch dynamic index field, " + - "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", - TypeConversions.fromLegacyInfoToDataType(indexFieldType))); - } - } - // general dynamic index pattern - return new IndexGeneratorBase(index) { - @Override - public String generate(Row row) { - Object indexField = row.getField(indexFieldPos); - return indexPrefix.concat(indexField == null ? "null" : indexField.toString()).concat(indexSuffix); - } - }; - } + // time extract dynamic index pattern + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldType); + // DataTypes.SQL_TIMESTAMP + if (indexFieldType == Types.LOCAL_DATE_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalDateTime indexField = (LocalDateTime) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else if (indexFieldType == Types.SQL_TIMESTAMP) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Timestamp indexField = (Timestamp) row.getField(indexFieldPos); + String indexFieldValueStr = + indexField.toLocalDateTime().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } + // DataTypes.SQL_DATE + else if (indexFieldType == Types.LOCAL_DATE) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalDate indexField = (LocalDate) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else if (indexFieldType == Types.SQL_DATE) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Date indexField = (Date) row.getField(indexFieldPos); + String indexFieldValueStr = + indexField.toLocalDate().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } // DataTypes.TIME + else if (indexFieldType == Types.LOCAL_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + LocalTime indexField = (LocalTime) row.getField(indexFieldPos); + String indexFieldValueStr = indexField.format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else if (indexFieldType == Types.SQL_TIME) { + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(Row row) { + Time indexField = (Time) row.getField(indexFieldPos); + String indexFieldValueStr = + indexField.toLocalTime().format(dateTimeFormatter); + return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); + } + }; + } else { + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + TypeConversions.fromLegacyInfoToDataType(indexFieldType))); + } + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(Row row) { + Object indexField = row.getField(indexFieldPos); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } - /** - * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field type - * ans parse index format from pattern. - */ - private static class IndexHelper { - private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); - private static final Pattern dynamicIndexTimeExtractPattern = Pattern.compile(".*\\{.+\\|.*\\}.*"); - private static final List supportedTypes = new ArrayList<>(); - private static final Map defaultFormats = new HashMap<>(); + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + private static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); - static { - //time related types - supportedTypes.add(Types.LOCAL_DATE_TIME); - supportedTypes.add(Types.SQL_TIMESTAMP); - supportedTypes.add(Types.LOCAL_DATE); - supportedTypes.add(Types.SQL_DATE); - supportedTypes.add(Types.LOCAL_TIME); - supportedTypes.add(Types.SQL_TIME); - //general types - supportedTypes.add(Types.STRING); - supportedTypes.add(Types.SHORT); - supportedTypes.add(Types.INT); - supportedTypes.add(Types.LONG); - } + static { + // time related types + supportedTypes.add(Types.LOCAL_DATE_TIME); + supportedTypes.add(Types.SQL_TIMESTAMP); + supportedTypes.add(Types.LOCAL_DATE); + supportedTypes.add(Types.SQL_DATE); + supportedTypes.add(Types.LOCAL_TIME); + supportedTypes.add(Types.SQL_TIME); + // general types + supportedTypes.add(Types.STRING); + supportedTypes.add(Types.SHORT); + supportedTypes.add(Types.INT); + supportedTypes.add(Types.LONG); + } - static { - defaultFormats.put(Types.LOCAL_DATE_TIME, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(Types.SQL_TIMESTAMP, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(Types.LOCAL_DATE, "yyyy_MM_dd"); - defaultFormats.put(Types.SQL_DATE, "yyyy_MM_dd"); - defaultFormats.put(Types.LOCAL_TIME, "HH_mm_ss"); - defaultFormats.put(Types.SQL_TIME, "HH_mm_ss"); - } + static { + defaultFormats.put(Types.LOCAL_DATE_TIME, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(Types.SQL_TIMESTAMP, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(Types.LOCAL_DATE, "yyyy_MM_dd"); + defaultFormats.put(Types.SQL_DATE, "yyyy_MM_dd"); + defaultFormats.put(Types.LOCAL_TIME, "HH_mm_ss"); + defaultFormats.put(Types.SQL_TIME, "HH_mm_ss"); + } - /** - * Validate the index field Type. - */ - void validateIndexFieldType(TypeInformation indexTypeInfo) { - if (!supportedTypes.contains(indexTypeInfo)) { - throw new IllegalArgumentException(String.format("Unsupported type %s of index field, " + - "Supported types are: %s", indexTypeInfo, supportedTypes)); - } - } + /** Validate the index field Type. */ + void validateIndexFieldType(TypeInformation indexTypeInfo) { + if (!supportedTypes.contains(indexTypeInfo)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + indexTypeInfo, supportedTypes)); + } + } - /** - * Get the default date format. - */ - String getDefaultFormat(TypeInformation indexTypeInfo) { - return defaultFormats.get(indexTypeInfo); - } + /** Get the default date format. */ + String getDefaultFormat(TypeInformation indexTypeInfo) { + return defaultFormats.get(indexTypeInfo); + } - /** - * Check general dynamic index is enabled or not by index pattern. - */ - boolean checkIsDynamicIndex(String index) { - final Matcher matcher = dynamicIndexPattern.matcher(index); - int count = 0; - while (matcher.find()) { - count++; - } - if (count > 1) { - throw new TableException(String.format("Chaining dynamic index pattern %s is not supported," + - " only support single dynamic index pattern.", index)); - } - return count == 1; - } + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } - /** - * Check time extract dynamic index is enabled or not by index pattern. - */ - boolean checkIsDynamicIndexWithFormat(String index) { - return dynamicIndexTimeExtractPattern.matcher(index).matches(); - } + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } - /** - * Extract dynamic index pattern string from index pattern string. - */ - String extractDynamicIndexPatternStr(String index) { - int start = index.indexOf("{"); - int end = index.lastIndexOf("}"); - return index.substring(start, end + 1); - } + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } - /** - * Extract index field position in a fieldNames, return the field position. - */ - int extractIndexFieldPos(String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { - List fieldList = Arrays.asList(fieldNames); - String indexFieldName; - if (isDynamicIndexWithFormat) { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); - } else { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); - } - if (!fieldList.contains(indexFieldName)) { - throw new TableException(String.format("Unknown field '%s' in index pattern '%s', please check the field name.", - indexFieldName, index)); - } - return fieldList.indexOf(indexFieldName); - } + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } - /** - * Extract dateTime format by the date format that extracted from index pattern string. - */ - private String extractDateFormat(String index, TypeInformation indexTypeInfo) { - String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); - if ("".equals(format)) { - format = getDefaultFormat(indexTypeInfo); - } - return format; - } - } + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, TypeInformation indexTypeInfo) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(indexTypeInfo); + } + return format; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java index 054ee941..b0564fe3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java @@ -20,16 +20,14 @@ import org.apache.flink.types.Row; -/** - * A static {@link IndexGenerator} which generate fixed index name. - */ +/** A static {@link IndexGenerator} which generate fixed index name. */ public class StaticIndexGenerator extends IndexGeneratorBase { - public StaticIndexGenerator(String index) { - super(index); - } + public StaticIndexGenerator(String index) { + super(index); + } - public String generate(Row row) { - return index; - } + public String generate(Row row) { + return index; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java index 864d7fab..6c22cf3b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java @@ -22,22 +22,20 @@ import java.time.format.DateTimeFormatter; -/** - * Abstract class for time related {@link IndexGenerator}. - */ +/** Abstract class for time related {@link IndexGenerator}. */ @Internal abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { - private final String dateTimeFormat; - protected transient DateTimeFormatter dateTimeFormatter; + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; - public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { - super(index); - this.dateTimeFormat = dateTimeFormat; - } + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } - @Override - public void open() { - this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); - } + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index 25b5d6b9..e1ffab77 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -40,128 +40,129 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; -/** - * Accessor methods to elasticsearch options. - */ +/** Accessor methods to elasticsearch options. */ @Internal class ElasticsearchConfiguration { - protected final ReadableConfig config; - private final ClassLoader classLoader; - - ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { - this.config = config; - this.classLoader = classLoader; - } - - public ActionRequestFailureHandler getFailureHandler() { - final ActionRequestFailureHandler failureHandler; - String value = config.get(FAILURE_HANDLER_OPTION); - switch (value.toUpperCase()) { - case "FAIL": - failureHandler = new NoOpFailureHandler(); - break; - case "IGNORE": - failureHandler = new IgnoringFailureHandler(); - break; - case "RETRY-REJECTED": - failureHandler = new RetryRejectedExecutionFailureHandler(); - break; - default: - try { - Class failureHandlerClass = Class.forName(value, false, classLoader); - failureHandler = (ActionRequestFailureHandler) InstantiationUtil.instantiate(failureHandlerClass); - } catch (ClassNotFoundException e) { - throw new ValidationException("Could not instantiate the failure handler class: " + value, e); - } - break; - } - return failureHandler; - } - - public String getDocumentType() { - return config.get(ElasticsearchOptions.DOCUMENT_TYPE_OPTION); - } - - public int getBulkFlushMaxActions() { - int maxActions = config.get(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return maxActions == 0 ? -1 : maxActions; - } - - public long getBulkFlushMaxByteSize() { - long maxSize = config.get(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return maxSize == 0 ? -1 : maxSize; - } - - public long getBulkFlushInterval() { - long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return interval == 0 ? -1 : interval; - } - - public Optional getUsername() { - return config.getOptional(USERNAME_OPTION); - } - - public Optional getPassword() { - return config.getOptional(PASSWORD_OPTION); - } - - public boolean isBulkFlushBackoffEnabled() { - return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) != ElasticsearchOptions.BackOffType.DISABLED; - } - - public Optional getBulkFlushBackoffType() { - switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { - case CONSTANT: - return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); - case EXPONENTIAL: - return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - default: - return Optional.empty(); - } - } - - public Optional getBulkFlushBackoffRetries() { - return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); - } - - public Optional getBulkFlushBackoffDelay() { - return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); - } - - public boolean isDisableFlushOnCheckpoint() { - return !config.get(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION); - } - - public String getIndex() { - return config.get(ElasticsearchOptions.INDEX_OPTION); - } - - public String getKeyDelimiter() { - return config.get(ElasticsearchOptions.KEY_DELIMITER_OPTION); - } - - public Optional getPathPrefix() { - return config.getOptional(ElasticsearchOptions.CONNECTION_PATH_PREFIX); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; - return Objects.equals(config, that.config) && - Objects.equals(classLoader, that.classLoader); - } - - @Override - public int hashCode() { - return Objects.hash(config, classLoader); - } + protected final ReadableConfig config; + private final ClassLoader classLoader; + + ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { + this.config = config; + this.classLoader = classLoader; + } + + public ActionRequestFailureHandler getFailureHandler() { + final ActionRequestFailureHandler failureHandler; + String value = config.get(FAILURE_HANDLER_OPTION); + switch (value.toUpperCase()) { + case "FAIL": + failureHandler = new NoOpFailureHandler(); + break; + case "IGNORE": + failureHandler = new IgnoringFailureHandler(); + break; + case "RETRY-REJECTED": + failureHandler = new RetryRejectedExecutionFailureHandler(); + break; + default: + try { + Class failureHandlerClass = Class.forName(value, false, classLoader); + failureHandler = + (ActionRequestFailureHandler) + InstantiationUtil.instantiate(failureHandlerClass); + } catch (ClassNotFoundException e) { + throw new ValidationException( + "Could not instantiate the failure handler class: " + value, e); + } + break; + } + return failureHandler; + } + + public String getDocumentType() { + return config.get(ElasticsearchOptions.DOCUMENT_TYPE_OPTION); + } + + public int getBulkFlushMaxActions() { + int maxActions = config.get(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxActions == 0 ? -1 : maxActions; + } + + public long getBulkFlushMaxByteSize() { + long maxSize = config.get(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxSize == 0 ? -1 : maxSize; + } + + public long getBulkFlushInterval() { + long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return interval == 0 ? -1 : interval; + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public boolean isBulkFlushBackoffEnabled() { + return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) + != ElasticsearchOptions.BackOffType.DISABLED; + } + + public Optional getBulkFlushBackoffType() { + switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { + case CONSTANT: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); + case EXPONENTIAL: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + default: + return Optional.empty(); + } + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { + return !config.get(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION); + } + + public String getIndex() { + return config.get(ElasticsearchOptions.INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(ElasticsearchOptions.KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(ElasticsearchOptions.CONNECTION_PATH_PREFIX); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; + return Objects.equals(config, that.config) && Objects.equals(classLoader, that.classLoader); + } + + @Override + public int hashCode() { + return Objects.hash(config, classLoader); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index 9cd781a5..7f437f95 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -33,112 +33,119 @@ * Options for {@link org.apache.flink.table.factories.DynamicTableSinkFactory} for Elasticsearch. */ public class ElasticsearchOptions { - /** - * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with - * {@code DISABLED} option. - */ - public enum BackOffType { - DISABLED, - CONSTANT, - EXPONENTIAL - } + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code + * DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } - public static final ConfigOption> HOSTS_OPTION = - ConfigOptions.key("hosts") - .stringType() - .asList() - .noDefaultValue() - .withDescription("Elasticsearch hosts to connect to."); - public static final ConfigOption INDEX_OPTION = - ConfigOptions.key("index") - .stringType() - .noDefaultValue() - .withDescription("Elasticsearch index for every record."); - public static final ConfigOption DOCUMENT_TYPE_OPTION = - ConfigOptions.key("document-type") - .stringType() - .noDefaultValue() - .withDescription("Elasticsearch document type."); - public static final ConfigOption PASSWORD_OPTION = - ConfigOptions.key("password") - .stringType() - .noDefaultValue() - .withDescription("Password used to connect to Elasticsearch instance."); - public static final ConfigOption USERNAME_OPTION = - ConfigOptions.key("username") - .stringType() - .noDefaultValue() - .withDescription("Username used to connect to Elasticsearch instance."); - public static final ConfigOption KEY_DELIMITER_OPTION = - ConfigOptions.key("document-id.key-delimiter") - .stringType() - .defaultValue("_") - .withDescription("Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); - public static final ConfigOption FAILURE_HANDLER_OPTION = - ConfigOptions.key("failure-handler") - .stringType() - .defaultValue("fail") - .withDescription(Description.builder() - .text("Failure handling strategy in case a request to Elasticsearch fails") - .list( - text("\"fail\" (throws an exception if a request fails and thus causes a job failure),"), - text("\"ignore\" (ignores failures and drops the request),"), - text("\"retry_rejected\" (re-adds requests that have failed due to queue capacity saturation),"), - text("\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) - .build()); - public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = - ConfigOptions.key("sink.flush-on-checkpoint") - .booleanType() - .defaultValue(true) - .withDescription("Disables flushing on checkpoint"); - public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = - ConfigOptions.key("sink.bulk-flush.max-actions") - .intType() - .defaultValue(1000) - .withDescription("Maximum number of actions to buffer for each bulk request."); - public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = - ConfigOptions.key("sink.bulk-flush.max-size") - .memoryType() - .defaultValue(MemorySize.parse("2mb")) - .withDescription("Maximum size of buffered actions per bulk request"); - public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = - ConfigOptions.key("sink.bulk-flush.interval") - .durationType() - .defaultValue(Duration.ofSeconds(1)) - .withDescription("Bulk flush interval"); - public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.strategy") - .enumType(BackOffType.class) - .defaultValue(BackOffType.DISABLED) - .withDescription("Backoff strategy"); - public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.max-retries") - .intType() - .noDefaultValue() - .withDescription("Maximum number of retries."); - public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.delay") - .durationType() - .noDefaultValue() - .withDescription("Delay between each backoff attempt."); - public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = - ConfigOptions.key("connection.max-retry-timeout") - .durationType() - .noDefaultValue() - .withDescription("Maximum timeout between retries."); - public static final ConfigOption CONNECTION_PATH_PREFIX = - ConfigOptions.key("connection.path-prefix") - .stringType() - .noDefaultValue() - .withDescription("Prefix string to be added to every REST communication."); - public static final ConfigOption FORMAT_OPTION = - ConfigOptions.key("format") - .stringType() - .defaultValue("json") - .withDescription("Elasticsearch connector requires to specify a format.\n" + - "The format must produce a valid json document. \n" + - "By default uses built-in 'json' format. Please refer to Table Formats section for more details."); + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + public static final ConfigOption FAILURE_HANDLER_OPTION = + ConfigOptions.key("failure-handler") + .stringType() + .defaultValue("fail") + .withDescription( + Description.builder() + .text( + "Failure handling strategy in case a request to Elasticsearch fails") + .list( + text( + "\"fail\" (throws an exception if a request fails and thus causes a job failure),"), + text( + "\"ignore\" (ignores failures and drops the request),"), + text( + "\"retry_rejected\" (re-adds requests that have failed due to queue capacity saturation),"), + text( + "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) + .build()); + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = + ConfigOptions.key("sink.flush-on-checkpoint") + .booleanType() + .defaultValue(true) + .withDescription("Disables flushing on checkpoint"); + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(BackOffType.class) + .defaultValue(BackOffType.DISABLED) + .withDescription("Backoff strategy"); + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = + ConfigOptions.key("connection.max-retry-timeout") + .durationType() + .noDefaultValue() + .withDescription("Maximum timeout between retries."); + public static final ConfigOption CONNECTION_PATH_PREFIX = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "Elasticsearch connector requires to specify a format.\n" + + "The format must produce a valid json document. \n" + + "By default uses built-in 'json' format. Please refer to Table Formats section for more details."); - private ElasticsearchOptions() { - } + private ElasticsearchOptions() {} } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java index b5caa765..d0657e61 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -33,61 +33,65 @@ import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; -/** - * Utility methods for validating Elasticsearch properties. - */ +/** Utility methods for validating Elasticsearch properties. */ @Internal class ElasticsearchValidationUtils { - private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); - static { - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); - } + static { + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + } - /** - * Checks that the table does not have primary key defined on illegal types. - * In Elasticsearch the primary key is used to calculate the Elasticsearch document id, - * which is a string of up to 512 bytes. It cannot have whitespaces. As of now it is calculated - * by concatenating the fields. Certain types do not have a good string representation to be used - * in this scenario. The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and - * {@link LogicalTypeRoot#RAW} type. - */ - public static void validatePrimaryKey(TableSchema schema) { - schema.getPrimaryKey().ifPresent( - key -> { - List illegalTypes = key.getColumns() - .stream() - .map(fieldName -> { - LogicalType logicalType = schema.getFieldDataType(fieldName).get().getLogicalType(); - if (hasRoot(logicalType, LogicalTypeRoot.DISTINCT_TYPE)) { - return ((DistinctType) logicalType).getSourceType().getTypeRoot(); - } else { - return logicalType.getTypeRoot(); - } - }) - .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) - .collect(Collectors.toList()); + /** + * Checks that the table does not have primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(TableSchema schema) { + schema.getPrimaryKey() + .ifPresent( + key -> { + List illegalTypes = + key.getColumns().stream() + .map( + fieldName -> { + LogicalType logicalType = + schema.getFieldDataType(fieldName) + .get() + .getLogicalType(); + if (hasRoot( + logicalType, + LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) + .collect(Collectors.toList()); - if (!illegalTypes.isEmpty()) { - throw new ValidationException( - String.format( - "The table has a primary key on columns of illegal types: %s.\n" + - " Elasticsearch sink does not support primary keys on columns of types: %s.", - illegalTypes, - ILLEGAL_PRIMARY_KEY_TYPES)); - } - } - ); - } + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.\n" + + " Elasticsearch sink does not support primary keys on columns of types: %s.", + illegalTypes, ILLEGAL_PRIMARY_KEY_TYPES)); + } + }); + } - private ElasticsearchValidationUtils() { - } + private ElasticsearchValidationUtils() {} } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java index f5faf84c..636f3409 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -24,19 +24,16 @@ import java.io.Serializable; -/** - * This interface is responsible to generate index name from given {@link Row} record. - */ +/** This interface is responsible to generate index name from given {@link Row} record. */ @Internal interface IndexGenerator extends Serializable { - /** - * Initialize the index generator, this will be called only once before {@link #generate(RowData)} is called. - */ - default void open() {} + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} - /** - * Generate index name according the the given row. - */ - String generate(RowData row); + /** Generate index name according the the given row. */ + String generate(RowData row); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java index 5df3efdf..adfcaa46 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java @@ -22,33 +22,31 @@ import java.util.Objects; -/** - * Base class for {@link IndexGenerator}. - */ +/** Base class for {@link IndexGenerator}. */ @Internal public abstract class IndexGeneratorBase implements IndexGenerator { - private static final long serialVersionUID = 1L; - protected final String index; - - public IndexGeneratorBase(String index) { - this.index = index; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof IndexGeneratorBase)) { - return false; - } - IndexGeneratorBase that = (IndexGeneratorBase) o; - return index.equals(that.index); - } - - @Override - public int hashCode() { - return Objects.hash(index); - } + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java index 692b1fe3..bf7ac64a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -47,230 +47,229 @@ * *

Flink supports both static index and dynamic index. * - *

If you want to have a static index, this option value should be a plain string, e.g. 'myusers', - * all the records will be consistently written into "myusers" index. + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. * - *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in the - * record to dynamically generate a target index. You can also use '{field_name|date_format_string}' to - * convert a field value of TIMESTAMP/DATE/TIME type into the format specified by date_format_string. The - * date_format_string is compatible with {@link java.text.SimpleDateFormat}. For example, if the option - * value is 'myusers_{log_ts|yyyy-MM-dd}', then a record with log_ts field value 2020-03-27 12:25:55 will - * be written into "myusers_2020-03-27" index. + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. */ @Internal final class IndexGeneratorFactory { - private IndexGeneratorFactory() {} + private IndexGeneratorFactory() {} - public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { - final IndexHelper indexHelper = new IndexHelper(); - if (indexHelper.checkIsDynamicIndex(index)) { - return createRuntimeIndexGenerator(index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); - } else { - return new StaticIndexGenerator(index); - } - } + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } - interface DynamicFormatter extends Serializable { - String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); - } + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } - private static IndexGenerator createRuntimeIndexGenerator( - String index, - String[] fieldNames, - DataType[] fieldTypes, - IndexHelper indexHelper) { - final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); - final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); - final String indexSuffix = index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + private static IndexGenerator createRuntimeIndexGenerator( + String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); - final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); - final int indexFieldPos = indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); - final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); - final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); - // validate index field type - indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); - // time extract dynamic index pattern - final RowData.FieldGetter fieldGetter = RowData.createFieldGetter(indexFieldType, indexFieldPos); + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); - if (isDynamicIndexWithFormat) { - final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); - DynamicFormatter formatFunction = createFormatFunction( - indexFieldType, - indexFieldLogicalTypeRoot); + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(RowData row) { - Object fieldOrNull = fieldGetter.getFieldOrNull(row); - final String formattedField; - // TODO we can possibly optimize it to use the nullability of the field - if (fieldOrNull != null) { - formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); - } else { - formattedField = "null"; - } - return indexPrefix.concat(formattedField).concat(indexSuffix); - } - }; - } - // general dynamic index pattern - return new IndexGeneratorBase(index) { - @Override - public String generate(RowData row) { - Object indexField = fieldGetter.getFieldOrNull(row); - return indexPrefix.concat(indexField == null ? "null" : indexField.toString()).concat(indexSuffix); - } - }; - } + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } - private static DynamicFormatter createFormatFunction( - LogicalType indexFieldType, - LogicalTypeRoot indexFieldLogicalTypeRoot) { - switch (indexFieldLogicalTypeRoot) { - case DATE: - return (value, dateTimeFormatter) -> { - Integer indexField = (Integer) value; - return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); - }; - case TIME_WITHOUT_TIME_ZONE: - return (value, dateTimeFormatter) -> { - Integer indexField = (Integer) value; - return LocalTime.ofNanoOfDay(indexField * 1_000_000L) - .format(dateTimeFormatter); - }; - case TIMESTAMP_WITHOUT_TIME_ZONE: - return (value, dateTimeFormatter) -> { - TimestampData indexField = (TimestampData) value; - return indexField.toLocalDateTime().format(dateTimeFormatter); - }; - case TIMESTAMP_WITH_TIME_ZONE: - throw new UnsupportedOperationException("TIMESTAMP_WITH_TIME_ZONE is not supported yet"); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return (value, dateTimeFormatter) -> { - TimestampData indexField = (TimestampData) value; - return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); - }; - default: - throw new TableException(String.format( - "Unsupported type '%s' found in Elasticsearch dynamic index field, " + - "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", - indexFieldType)); - } - } + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } - /** - * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field type - * ans parse index format from pattern. - */ - private static class IndexHelper { - private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); - private static final Pattern dynamicIndexTimeExtractPattern = Pattern.compile(".*\\{.+\\|.*\\}.*"); - private static final List supportedTypes = new ArrayList<>(); - private static final Map defaultFormats = new HashMap<>(); + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + private static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); - static { - //time related types - supportedTypes.add(LogicalTypeRoot.DATE); - supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); - supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); - supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); - supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); - //general types - supportedTypes.add(LogicalTypeRoot.VARCHAR); - supportedTypes.add(LogicalTypeRoot.CHAR); - supportedTypes.add(LogicalTypeRoot.TINYINT); - supportedTypes.add(LogicalTypeRoot.INTEGER); - supportedTypes.add(LogicalTypeRoot.BIGINT); - } + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } - static { + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } - defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); - defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); - defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); - } + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } - /** - * Validate the index field Type. - */ - void validateIndexFieldType(LogicalTypeRoot logicalType) { - if (!supportedTypes.contains(logicalType)) { - throw new IllegalArgumentException(String.format("Unsupported type %s of index field, " + - "Supported types are: %s", logicalType, supportedTypes)); - } - } + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } - /** - * Get the default date format. - */ - String getDefaultFormat(LogicalTypeRoot logicalType) { - return defaultFormats.get(logicalType); - } + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } - /** - * Check general dynamic index is enabled or not by index pattern. - */ - boolean checkIsDynamicIndex(String index) { - final Matcher matcher = dynamicIndexPattern.matcher(index); - int count = 0; - while (matcher.find()) { - count++; - } - if (count > 1) { - throw new TableException(String.format("Chaining dynamic index pattern %s is not supported," + - " only support single dynamic index pattern.", index)); - } - return count == 1; - } + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } - /** - * Check time extract dynamic index is enabled or not by index pattern. - */ - boolean checkIsDynamicIndexWithFormat(String index) { - return dynamicIndexTimeExtractPattern.matcher(index).matches(); - } + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } - /** - * Extract dynamic index pattern string from index pattern string. - */ - String extractDynamicIndexPatternStr(String index) { - int start = index.indexOf("{"); - int end = index.lastIndexOf("}"); - return index.substring(start, end + 1); - } + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } - /** - * Extract index field position in a fieldNames, return the field position. - */ - int extractIndexFieldPos(String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { - List fieldList = Arrays.asList(fieldNames); - String indexFieldName; - if (isDynamicIndexWithFormat) { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); - } else { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); - } - if (!fieldList.contains(indexFieldName)) { - throw new TableException(String.format("Unknown field '%s' in index pattern '%s', please check the field name.", - indexFieldName, index)); - } - return fieldList.indexOf(indexFieldName); - } - - /** - * Extract dateTime format by the date format that extracted from index pattern string. - */ - private String extractDateFormat(String index, LogicalTypeRoot logicalType) { - String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); - if ("".equals(format)) { - format = getDefaultFormat(logicalType); - } - return format; - } - } + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java index db28ff19..ae7c522b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -35,97 +35,96 @@ import java.util.Map; import java.util.function.Function; -/** - * An extractor for a Elasticsearch key from a {@link RowData}. - */ +/** An extractor for a Elasticsearch key from a {@link RowData}. */ @Internal class KeyExtractor implements Function, Serializable { - private final FieldFormatter[] fieldFormatters; - private final String keyDelimiter; + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; - private interface FieldFormatter extends Serializable { - String format(RowData rowData); - } + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } - private KeyExtractor( - FieldFormatter[] fieldFormatters, - String keyDelimiter) { - this.fieldFormatters = fieldFormatters; - this.keyDelimiter = keyDelimiter; - } + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } - @Override - public String apply(RowData rowData) { - final StringBuilder builder = new StringBuilder(); - for (int i = 0; i < fieldFormatters.length; i++) { - if (i > 0) { - builder.append(keyDelimiter); - } - final String value = fieldFormatters[i].format(rowData); - builder.append(value); - } - return builder.toString(); - } + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } - private static class ColumnWithIndex { - public TableColumn column; - public int index; + private static class ColumnWithIndex { + public TableColumn column; + public int index; - public ColumnWithIndex(TableColumn column, int index) { - this.column = column; - this.index = index; - } + public ColumnWithIndex(TableColumn column, int index) { + this.column = column; + this.index = index; + } - public LogicalType getType() { - return column.getType().getLogicalType(); - } + public LogicalType getType() { + return column.getType().getLogicalType(); + } - public int getIndex() { - return index; - } - } + public int getIndex() { + return index; + } + } - public static Function createKeyExtractor( - TableSchema schema, - String keyDelimiter) { - return schema.getPrimaryKey().map(key -> { - Map namesToColumns = new HashMap<>(); - List tableColumns = schema.getTableColumns(); - for (int i = 0; i < schema.getFieldCount(); i++) { - TableColumn column = tableColumns.get(i); - namesToColumns.put(column.getName(), new ColumnWithIndex(column, i)); - } + public static Function createKeyExtractor( + TableSchema schema, String keyDelimiter) { + return schema.getPrimaryKey() + .map( + key -> { + Map namesToColumns = new HashMap<>(); + List tableColumns = schema.getTableColumns(); + for (int i = 0; i < schema.getFieldCount(); i++) { + TableColumn column = tableColumns.get(i); + namesToColumns.put( + column.getName(), new ColumnWithIndex(column, i)); + } - FieldFormatter[] fieldFormatters = key.getColumns() - .stream() - .map(namesToColumns::get) - .map(column -> toFormatter(column.index, column.getType())) - .toArray(FieldFormatter[]::new); + FieldFormatter[] fieldFormatters = + key.getColumns().stream() + .map(namesToColumns::get) + .map( + column -> + toFormatter( + column.index, column.getType())) + .toArray(FieldFormatter[]::new); - return (Function) new KeyExtractor( - fieldFormatters, - keyDelimiter - ); - }).orElseGet(() -> (Function & Serializable) (row) -> null); - } + return (Function) + new KeyExtractor(fieldFormatters, keyDelimiter); + }) + .orElseGet(() -> (Function & Serializable) (row) -> null); + } - private static FieldFormatter toFormatter(int index, LogicalType type) { - switch (type.getTypeRoot()) { - case DATE: - return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); - case TIME_WITHOUT_TIME_ZONE: - return (row) -> LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); - case INTERVAL_YEAR_MONTH: - return (row) -> Period.ofDays(row.getInt(index)).toString(); - case INTERVAL_DAY_TIME: - return (row) -> Duration.ofMillis(row.getLong(index)).toString(); - case DISTINCT_TYPE: - return toFormatter(index, ((DistinctType) type).getSourceType()); - default: - RowData.FieldGetter fieldGetter = RowData.createFieldGetter( - type, - index); - return (row) -> fieldGetter.getFieldOrNull(row).toString(); - } - } + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java index 35d69eb6..f5b24180 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java @@ -29,39 +29,26 @@ import java.io.Serializable; -/** - * For version-agnostic creating of {@link ActionRequest}s. - */ +/** For version-agnostic creating of {@link ActionRequest}s. */ @Internal interface RequestFactory extends Serializable { - /** - * Creates an update request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document); + /** + * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, String docType, String key, XContentType contentType, byte[] document); - /** - * Creates an index request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - IndexRequest createIndexRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document); + /** + * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, String docType, String key, XContentType contentType, byte[] document); - /** - * Creates a delete request to be added to a {@link RequestIndexer}. - * Note: the type field has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - DeleteRequest createDeleteRequest( - String index, - String docType, - String key); + /** + * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest(String index, String docType, String key); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java index 76701d8b..7fb1e4d4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -38,110 +38,103 @@ import java.util.Objects; import java.util.function.Function; -/** - * Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. - */ +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ @Internal class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { - private static final long serialVersionUID = 1L; + private static final long serialVersionUID = 1L; - private final IndexGenerator indexGenerator; - private final String docType; - private final SerializationSchema serializationSchema; - private final XContentType contentType; - private final RequestFactory requestFactory; - private final Function createKey; + private final IndexGenerator indexGenerator; + private final String docType; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final Function createKey; - public RowElasticsearchSinkFunction( - IndexGenerator indexGenerator, - @Nullable String docType, // this is deprecated in es 7+ - SerializationSchema serializationSchema, - XContentType contentType, - RequestFactory requestFactory, - Function createKey) { - this.indexGenerator = Preconditions.checkNotNull(indexGenerator); - this.docType = docType; - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - this.createKey = Preconditions.checkNotNull(createKey); - } + public RowElasticsearchSinkFunction( + IndexGenerator indexGenerator, + @Nullable String docType, // this is deprecated in es 7+ + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + Function createKey) { + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = docType; + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.createKey = Preconditions.checkNotNull(createKey); + } - @Override - public void open() { - indexGenerator.open(); - } + @Override + public void open() { + indexGenerator.open(); + } - @Override - public void process( - RowData element, - RuntimeContext ctx, - RequestIndexer indexer) { - switch (element.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - processUpsert(element, indexer); - break; - case UPDATE_BEFORE: - case DELETE: - processDelete(element, indexer); - break; - default: - throw new TableException("Unsupported message kind: " + element.getRowKind()); - } - } + @Override + public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } - private void processUpsert(RowData row, RequestIndexer indexer) { - final byte[] document = serializationSchema.serialize(row); - final String key = createKey.apply(row); - if (key != null) { - final UpdateRequest updateRequest = requestFactory.createUpdateRequest( - indexGenerator.generate(row), - docType, - key, - contentType, - document); - indexer.add(updateRequest); - } else { - final IndexRequest indexRequest = requestFactory.createIndexRequest( - indexGenerator.generate(row), - docType, - key, - contentType, - document); - indexer.add(indexRequest); - } - } + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + requestFactory.createUpdateRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + requestFactory.createIndexRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(indexRequest); + } + } - private void processDelete(RowData row, RequestIndexer indexer) { - final String key = createKey.apply(row); - final DeleteRequest deleteRequest = requestFactory.createDeleteRequest( - indexGenerator.generate(row), - docType, - key); - indexer.add(deleteRequest); - } + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = + requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key); + indexer.add(deleteRequest); + } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; - return Objects.equals(indexGenerator, that.indexGenerator) && - Objects.equals(docType, that.docType) && - Objects.equals(serializationSchema, that.serializationSchema) && - contentType == that.contentType && - Objects.equals(requestFactory, that.requestFactory) && - Objects.equals(createKey, that.createKey); - } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) + && Objects.equals(docType, that.docType) + && Objects.equals(serializationSchema, that.serializationSchema) + && contentType == that.contentType + && Objects.equals(requestFactory, that.requestFactory) + && Objects.equals(createKey, that.createKey); + } - @Override - public int hashCode() { - return Objects.hash(indexGenerator, docType, serializationSchema, contentType, requestFactory, createKey); - } + @Override + public int hashCode() { + return Objects.hash( + indexGenerator, + docType, + serializationSchema, + contentType, + requestFactory, + createKey); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java index 196b64ce..1ffcac48 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java @@ -21,17 +21,15 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; -/** - * A static {@link IndexGenerator} which generate fixed index name. - */ +/** A static {@link IndexGenerator} which generate fixed index name. */ @Internal final class StaticIndexGenerator extends IndexGeneratorBase { - public StaticIndexGenerator(String index) { - super(index); - } + public StaticIndexGenerator(String index) { + super(index); + } - public String generate(RowData row) { - return index; - } + public String generate(RowData row) { + return index; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java index 11eede4b..e311b09a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java @@ -27,28 +27,27 @@ import java.util.ArrayList; import java.util.List; -/** - * Suite of utility methods for Elasticsearch. - */ +/** Suite of utility methods for Elasticsearch. */ @Internal public class ElasticsearchUtils { - /** - * Utility method to convert a {@link List} of {@link InetSocketAddress} to Elasticsearch {@link TransportAddress}. - * - * @param inetSocketAddresses The list of {@link InetSocketAddress} to convert. - */ - public static List convertInetSocketAddresses(List inetSocketAddresses) { - if (inetSocketAddresses == null) { - return null; - } else { - List converted; - converted = new ArrayList<>(inetSocketAddresses.size()); - for (InetSocketAddress address : inetSocketAddresses) { - converted.add(new InetSocketTransportAddress(address)); - } - return converted; - } - } - + /** + * Utility method to convert a {@link List} of {@link InetSocketAddress} to Elasticsearch {@link + * TransportAddress}. + * + * @param inetSocketAddresses The list of {@link InetSocketAddress} to convert. + */ + public static List convertInetSocketAddresses( + List inetSocketAddresses) { + if (inetSocketAddresses == null) { + return null; + } else { + List converted; + converted = new ArrayList<>(inetSocketAddresses.size()); + for (InetSocketAddress address : inetSocketAddresses) { + converted.add(new InetSocketTransportAddress(address)); + } + return converted; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java index a3644b45..058d4c76 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java @@ -24,16 +24,15 @@ import org.elasticsearch.action.ActionRequest; -/** - * Ignores all kinds of failures and drops the affected {@link ActionRequest}. - */ +/** Ignores all kinds of failures and drops the affected {@link ActionRequest}. */ @Internal public class IgnoringFailureHandler implements ActionRequestFailureHandler { - private static final long serialVersionUID = 1662846593501L; + private static final long serialVersionUID = 1662846593501L; - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { - // ignore failure - } + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { + // ignore failure + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java index c076fc8e..cd6e5063 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java @@ -25,30 +25,30 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** - * An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. - */ +/** An {@link ActionRequestFailureHandler} that simply fails the sink on any failures. */ @Internal public class NoOpFailureHandler implements ActionRequestFailureHandler { - private static final long serialVersionUID = 737941343410827885L; + private static final long serialVersionUID = 737941343410827885L; - private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class); + private static final Logger LOG = LoggerFactory.getLogger(NoOpFailureHandler.class); - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); - // simply fail the sink - throw failure; - } + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + // simply fail the sink + throw failure; + } - @Override - public boolean equals(Object o) { - return o instanceof NoOpFailureHandler; - } + @Override + public boolean equals(Object o) { + return o instanceof NoOpFailureHandler; + } - @Override - public int hashCode() { - return NoOpFailureHandler.class.hashCode(); - } + @Override + public int hashCode() { + return NoOpFailureHandler.class.hashCode(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index 98b58f94..7b0b861d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -29,26 +29,28 @@ import org.slf4j.LoggerFactory; /** - * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary - * {@link EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), + * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary {@link + * EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), * and fails for all other failures. */ @PublicEvolving public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { - private static final long serialVersionUID = -7423562912824511906L; - - private static final Logger LOG = LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class); - - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); - if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { - indexer.add(action); - } else { - // rethrow all other failures - throw failure; - } - } - + private static final long serialVersionUID = -7423562912824511906L; + + private static final Logger LOG = + LoggerFactory.getLogger(RetryRejectedExecutionFailureHandler.class); + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + LOG.error("Failed Elasticsearch item request: {}", failure.getMessage(), failure); + if (ExceptionUtils.findThrowable(failure, EsRejectedExecutionException.class).isPresent()) { + indexer.add(action); + } else { + // rethrow all other failures + throw failure; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java index ccbd4c10..710fe164 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java @@ -48,269 +48,278 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; -/** - * Connector descriptor for the Elasticsearch search engine. - */ +/** Connector descriptor for the Elasticsearch search engine. */ @PublicEvolving public class Elasticsearch extends ConnectorDescriptor { - private DescriptorProperties internalProperties = new DescriptorProperties(true); - private List hosts = new ArrayList<>(); + private DescriptorProperties internalProperties = new DescriptorProperties(true); + private List hosts = new ArrayList<>(); - /** - * Connector descriptor for the Elasticsearch search engine. - */ - public Elasticsearch() { - super(CONNECTOR_TYPE_VALUE_ELASTICSEARCH, 1, true); - } + /** Connector descriptor for the Elasticsearch search engine. */ + public Elasticsearch() { + super(CONNECTOR_TYPE_VALUE_ELASTICSEARCH, 1, true); + } - /** - * Sets the Elasticsearch version to be used. Required. - * - * @param version Elasticsearch version. E.g., "6". - */ - public Elasticsearch version(String version) { - internalProperties.putString(CONNECTOR_VERSION, version); - return this; - } + /** + * Sets the Elasticsearch version to be used. Required. + * + * @param version Elasticsearch version. E.g., "6". + */ + public Elasticsearch version(String version) { + internalProperties.putString(CONNECTOR_VERSION, version); + return this; + } - /** - * Adds an Elasticsearch host to connect to. Required. - * - *

Multiple hosts can be declared by calling this method multiple times. - * - * @param hostname connection hostname - * @param port connection port - * @param protocol connection protocol; e.g. "http" - */ - public Elasticsearch host(String hostname, int port, String protocol) { - final Host host = - new Host( - Preconditions.checkNotNull(hostname), - port, - Preconditions.checkNotNull(protocol)); - hosts.add(host); - return this; - } + /** + * Adds an Elasticsearch host to connect to. Required. + * + *

Multiple hosts can be declared by calling this method multiple times. + * + * @param hostname connection hostname + * @param port connection port + * @param protocol connection protocol; e.g. "http" + */ + public Elasticsearch host(String hostname, int port, String protocol) { + final Host host = + new Host( + Preconditions.checkNotNull(hostname), + port, + Preconditions.checkNotNull(protocol)); + hosts.add(host); + return this; + } - /** - * Declares the Elasticsearch index for every record. Required. - * - * @param index Elasticsearch index - */ - public Elasticsearch index(String index) { - internalProperties.putString(CONNECTOR_INDEX, index); - return this; - } + /** + * Declares the Elasticsearch index for every record. Required. + * + * @param index Elasticsearch index + */ + public Elasticsearch index(String index) { + internalProperties.putString(CONNECTOR_INDEX, index); + return this; + } - /** - * Declares the Elasticsearch document type for every record. Required. - * - * @param documentType Elasticsearch document type - */ - public Elasticsearch documentType(String documentType) { - internalProperties.putString(CONNECTOR_DOCUMENT_TYPE, documentType); - return this; - } + /** + * Declares the Elasticsearch document type for every record. Required. + * + * @param documentType Elasticsearch document type + */ + public Elasticsearch documentType(String documentType) { + internalProperties.putString(CONNECTOR_DOCUMENT_TYPE, documentType); + return this; + } - /** - * Sets a custom key delimiter in case the Elasticsearch ID needs to be constructed from - * multiple fields. Optional. - * - * @param keyDelimiter key delimiter; e.g., "$" would result in IDs "KEY1$KEY2$KEY3" - */ - public Elasticsearch keyDelimiter(String keyDelimiter) { - internalProperties.putString(CONNECTOR_KEY_DELIMITER, keyDelimiter); - return this; - } + /** + * Sets a custom key delimiter in case the Elasticsearch ID needs to be constructed from + * multiple fields. Optional. + * + * @param keyDelimiter key delimiter; e.g., "$" would result in IDs "KEY1$KEY2$KEY3" + */ + public Elasticsearch keyDelimiter(String keyDelimiter) { + internalProperties.putString(CONNECTOR_KEY_DELIMITER, keyDelimiter); + return this; + } - /** - * Sets a custom representation for null fields in keys. Optional. - * - * @param keyNullLiteral key null literal string; e.g. "N/A" would result in IDs "KEY1_N/A_KEY3" - */ - public Elasticsearch keyNullLiteral(String keyNullLiteral) { - internalProperties.putString(CONNECTOR_KEY_NULL_LITERAL, keyNullLiteral); - return this; - } + /** + * Sets a custom representation for null fields in keys. Optional. + * + * @param keyNullLiteral key null literal string; e.g. "N/A" would result in IDs "KEY1_N/A_KEY3" + */ + public Elasticsearch keyNullLiteral(String keyNullLiteral) { + internalProperties.putString(CONNECTOR_KEY_NULL_LITERAL, keyNullLiteral); + return this; + } - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy throws an exception if a request fails and thus causes a job failure. - */ - public Elasticsearch failureHandlerFail() { - internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL); - return this; - } + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy throws an exception if a request fails and thus causes a job failure. + */ + public Elasticsearch failureHandlerFail() { + internalProperties.putString( + CONNECTOR_FAILURE_HANDLER, + ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL); + return this; + } - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy ignores failures and drops the request. - */ - public Elasticsearch failureHandlerIgnore() { - internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE); - return this; - } + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy ignores failures and drops the request. + */ + public Elasticsearch failureHandlerIgnore() { + internalProperties.putString( + CONNECTOR_FAILURE_HANDLER, + ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE); + return this; + } - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy re-adds requests that have failed due to queue capacity saturation. - */ - public Elasticsearch failureHandlerRetryRejected() { - internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY); - return this; - } + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy re-adds requests that have failed due to queue capacity saturation. + */ + public Elasticsearch failureHandlerRetryRejected() { + internalProperties.putString( + CONNECTOR_FAILURE_HANDLER, + ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY); + return this; + } - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy allows for custom failure handling using a {@link ActionRequestFailureHandler}. - */ - public Elasticsearch failureHandlerCustom(Class failureHandlerClass) { - internalProperties.putString(CONNECTOR_FAILURE_HANDLER, ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM); - internalProperties.putClass(CONNECTOR_FAILURE_HANDLER_CLASS, failureHandlerClass); - return this; - } + /** + * Configures a failure handling strategy in case a request to Elasticsearch fails. + * + *

This strategy allows for custom failure handling using a {@link + * ActionRequestFailureHandler}. + */ + public Elasticsearch failureHandlerCustom( + Class failureHandlerClass) { + internalProperties.putString( + CONNECTOR_FAILURE_HANDLER, + ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM); + internalProperties.putClass(CONNECTOR_FAILURE_HANDLER_CLASS, failureHandlerClass); + return this; + } - /** - * Disables flushing on checkpoint. When disabled, a sink will not wait for all pending action - * requests to be acknowledged by Elasticsearch on checkpoints. - * - *

Note: If flushing on checkpoint is disabled, a Elasticsearch sink does NOT - * provide any strong guarantees for at-least-once delivery of action requests. - */ - public Elasticsearch disableFlushOnCheckpoint() { - internalProperties.putBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, false); - return this; - } + /** + * Disables flushing on checkpoint. When disabled, a sink will not wait for all pending action + * requests to be acknowledged by Elasticsearch on checkpoints. + * + *

Note: If flushing on checkpoint is disabled, a Elasticsearch sink does NOT provide any + * strong guarantees for at-least-once delivery of action requests. + */ + public Elasticsearch disableFlushOnCheckpoint() { + internalProperties.putBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, false); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum number of actions to buffer for each bulk request. - * - * @param maxActions the maximum number of actions to buffer per bulk request. - */ - public Elasticsearch bulkFlushMaxActions(int maxActions) { - internalProperties.putInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, maxActions); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum number of actions to buffer for each bulk request. + * + * @param maxActions the maximum number of actions to buffer per bulk request. + */ + public Elasticsearch bulkFlushMaxActions(int maxActions) { + internalProperties.putInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, maxActions); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum size of buffered actions per bulk request (using the syntax of {@link MemorySize}). - */ - public Elasticsearch bulkFlushMaxSize(String maxSize) { - internalProperties.putMemorySize(CONNECTOR_BULK_FLUSH_MAX_SIZE, MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum size of buffered actions per bulk request (using the syntax of {@link + * MemorySize}). + */ + public Elasticsearch bulkFlushMaxSize(String maxSize) { + internalProperties.putMemorySize( + CONNECTOR_BULK_FLUSH_MAX_SIZE, + MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the bulk flush interval (in milliseconds). - * - * @param interval bulk flush interval (in milliseconds). - */ - public Elasticsearch bulkFlushInterval(long interval) { - internalProperties.putLong(CONNECTOR_BULK_FLUSH_INTERVAL, interval); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the bulk flush interval (in milliseconds). + * + * @param interval bulk flush interval (in milliseconds). + */ + public Elasticsearch bulkFlushInterval(long interval) { + internalProperties.putLong(CONNECTOR_BULK_FLUSH_INTERVAL, interval); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets a constant backoff type to use when flushing bulk requests. - */ - public Elasticsearch bulkFlushBackoffConstant() { - internalProperties.putString( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets a constant backoff type to use when flushing bulk requests. + */ + public Elasticsearch bulkFlushBackoffConstant() { + internalProperties.putString( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets an exponential backoff type to use when flushing bulk requests. - */ - public Elasticsearch bulkFlushBackoffExponential() { - internalProperties.putString( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets an exponential backoff type to use when flushing bulk requests. + */ + public Elasticsearch bulkFlushBackoffExponential() { + internalProperties.putString( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. - * - *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} or - * {@link #bulkFlushBackoffExponential()}). - * - * @param maxRetries the maximum number of retries. - */ - public Elasticsearch bulkFlushBackoffMaxRetries(int maxRetries) { - internalProperties.putInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, maxRetries); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} + * or {@link #bulkFlushBackoffExponential()}). + * + * @param maxRetries the maximum number of retries. + */ + public Elasticsearch bulkFlushBackoffMaxRetries(int maxRetries) { + internalProperties.putInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, maxRetries); + return this; + } - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the amount of delay between each backoff attempt when flushing bulk requests (in milliseconds). - * - *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} or - * {@link #bulkFlushBackoffExponential()}). - * - * @param delay delay between each backoff attempt (in milliseconds). - */ - public Elasticsearch bulkFlushBackoffDelay(long delay) { - internalProperties.putLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, delay); - return this; - } + /** + * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. + * + *

Sets the amount of delay between each backoff attempt when flushing bulk requests (in + * milliseconds). + * + *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} + * or {@link #bulkFlushBackoffExponential()}). + * + * @param delay delay between each backoff attempt (in milliseconds). + */ + public Elasticsearch bulkFlushBackoffDelay(long delay) { + internalProperties.putLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, delay); + return this; + } - /** - * Sets connection properties to be used during REST communication to Elasticsearch. - * - *

Sets the maximum timeout (in milliseconds) in case of multiple retries of the same request. - * - * @param maxRetryTimeout maximum timeout (in milliseconds) - */ - public Elasticsearch connectionMaxRetryTimeout(int maxRetryTimeout) { - internalProperties.putInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, maxRetryTimeout); - return this; - } + /** + * Sets connection properties to be used during REST communication to Elasticsearch. + * + *

Sets the maximum timeout (in milliseconds) in case of multiple retries of the same + * request. + * + * @param maxRetryTimeout maximum timeout (in milliseconds) + */ + public Elasticsearch connectionMaxRetryTimeout(int maxRetryTimeout) { + internalProperties.putInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, maxRetryTimeout); + return this; + } - /** - * Sets connection properties to be used during REST communication to Elasticsearch. - * - *

Adds a path prefix to every REST communication. - * - * @param pathPrefix prefix string to be added to every REST communication - */ - public Elasticsearch connectionPathPrefix(String pathPrefix) { - internalProperties.putString(CONNECTOR_CONNECTION_PATH_PREFIX, pathPrefix); - return this; - } + /** + * Sets connection properties to be used during REST communication to Elasticsearch. + * + *

Adds a path prefix to every REST communication. + * + * @param pathPrefix prefix string to be added to every REST communication + */ + public Elasticsearch connectionPathPrefix(String pathPrefix) { + internalProperties.putString(CONNECTOR_CONNECTION_PATH_PREFIX, pathPrefix); + return this; + } - @Override - protected Map toConnectorProperties() { - final DescriptorProperties properties = new DescriptorProperties(); - properties.putProperties(internalProperties); + @Override + protected Map toConnectorProperties() { + final DescriptorProperties properties = new DescriptorProperties(); + properties.putProperties(internalProperties); - if (hosts.size() > 0) { - properties.putString( - CONNECTOR_HOSTS, - hosts.stream() - .map(Host::toString) - .collect(Collectors.joining(";"))); - } - return properties.asMap(); - } + if (hosts.size() > 0) { + properties.putString( + CONNECTOR_HOSTS, + hosts.stream().map(Host::toString).collect(Collectors.joining(";"))); + } + return properties.asMap(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java index b8f23318..da82fb7b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java @@ -34,151 +34,167 @@ import static org.apache.flink.table.descriptors.DescriptorProperties.noValidation; -/** - * The validator for {@link Elasticsearch}. - */ +/** The validator for {@link Elasticsearch}. */ @Internal public class ElasticsearchValidator extends ConnectorDescriptorValidator { - public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; - public static final String CONNECTOR_VERSION_VALUE_6 = "6"; - public static final String CONNECTOR_VERSION_VALUE_7 = "7"; - public static final String CONNECTOR_HOSTS = "connector.hosts"; - public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; - public static final String CONNECTOR_HOSTS_PORT = "port"; - public static final String CONNECTOR_HOSTS_PROTOCOL = "protocol"; - public static final String CONNECTOR_INDEX = "connector.index"; - public static final String CONNECTOR_DOCUMENT_TYPE = "connector.document-type"; - public static final String CONNECTOR_KEY_DELIMITER = "connector.key-delimiter"; - public static final String CONNECTOR_KEY_NULL_LITERAL = "connector.key-null-literal"; - public static final String CONNECTOR_FAILURE_HANDLER = "connector.failure-handler"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_FAIL = "fail"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE = "ignore"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_RETRY = "retry-rejected"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM = "custom"; - public static final String CONNECTOR_FAILURE_HANDLER_CLASS = "connector.failure-handler-class"; - public static final String CONNECTOR_FLUSH_ON_CHECKPOINT = "connector.flush-on-checkpoint"; - public static final String CONNECTOR_BULK_FLUSH_MAX_ACTIONS = "connector.bulk-flush.max-actions"; - public static final String CONNECTOR_BULK_FLUSH_MAX_SIZE = "connector.bulk-flush.max-size"; - public static final String CONNECTOR_BULK_FLUSH_INTERVAL = "connector.bulk-flush.interval"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE = "connector.bulk-flush.backoff.type"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED = "disabled"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT = "constant"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL = "exponential"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES = "connector.bulk-flush.backoff.max-retries"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_DELAY = "connector.bulk-flush.backoff.delay"; - public static final String CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT = "connector.connection-max-retry-timeout"; - public static final String CONNECTOR_CONNECTION_PATH_PREFIX = "connector.connection-path-prefix"; - - @Override - public void validate(DescriptorProperties properties) { - super.validate(properties); - properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); - validateVersion(properties); - validateHosts(properties); - validateGeneralProperties(properties); - validateFailureHandler(properties); - validateBulkFlush(properties); - validateConnectionProperties(properties); - } - - private void validateVersion(DescriptorProperties properties) { - properties.validateEnumValues( - CONNECTOR_VERSION, - false, - Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7)); - } - - private void validateHosts(DescriptorProperties properties) { - if (properties.containsKey(CONNECTOR_HOSTS)) { - validateAndParseHostsString(properties); - } else { - final Map> hostsValidators = new HashMap<>(); - hostsValidators.put(CONNECTOR_HOSTS_HOSTNAME, (key) -> properties.validateString(key, false, 1)); - hostsValidators.put(CONNECTOR_HOSTS_PORT, (key) -> properties.validateInt(key, false, 0, 65535)); - hostsValidators.put(CONNECTOR_HOSTS_PROTOCOL, (key) -> properties.validateString(key, false, 1)); - - properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); - } - } - - private void validateGeneralProperties(DescriptorProperties properties) { - properties.validateString(CONNECTOR_INDEX, false, 1); - properties.validateString(CONNECTOR_DOCUMENT_TYPE, false, 1); - properties.validateString(CONNECTOR_KEY_DELIMITER, true); - properties.validateString(CONNECTOR_KEY_NULL_LITERAL, true); - } - - private void validateFailureHandler(DescriptorProperties properties) { - final Map> failureHandlerValidators = new HashMap<>(); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_FAIL, noValidation()); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE, noValidation()); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_RETRY, noValidation()); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM, - key -> properties.validateString(CONNECTOR_FAILURE_HANDLER_CLASS, false, 1)); - properties.validateEnum(CONNECTOR_FAILURE_HANDLER, true, failureHandlerValidators); - } - - private void validateBulkFlush(DescriptorProperties properties) { - properties.validateBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, true); - properties.validateInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, true, 1); - properties.validateMemorySize(CONNECTOR_BULK_FLUSH_MAX_SIZE, true, 1024 * 1024); // only allow MB precision - properties.validateLong(CONNECTOR_BULK_FLUSH_INTERVAL, true, 0); - properties.validateEnumValues(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - true, - Arrays.asList( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED, - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT, - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL)); - properties.validateInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, true, 1); - properties.validateLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, true, 0); - } - - private void validateConnectionProperties(DescriptorProperties properties) { - properties.validateInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, true, 1); - properties.validateString(CONNECTOR_CONNECTION_PATH_PREFIX, true); - } - - /** - * Parse Hosts String to list. - * - *

Hosts String format was given as following: - * - *

-	 *     connector.hosts = http://host_name:9092;http://host_name:9093
-	 * 
- */ - public static List validateAndParseHostsString(DescriptorProperties descriptorProperties) { - final List hostList = new ArrayList<>(); - - descriptorProperties.validateString(CONNECTOR_HOSTS, false, 1); - final String hostsStr = descriptorProperties.getString(CONNECTOR_HOSTS); - - final String[] hosts = hostsStr.split(";"); - final String validationExceptionMessage = "Properties '" + CONNECTOR_HOSTS + "' format should " + - "follow the format 'http://host_name:port', but is '" + hostsStr + "'."; - - if (hosts.length == 0) { - throw new ValidationException(validationExceptionMessage); - } - for (String host : hosts) { - try { - final URL url = new URL(host); - final String protocol = url.getProtocol(); - final String hostName = url.getHost(); - final int hostPort = url.getPort(); - - if (StringUtils.isNullOrWhitespaceOnly(protocol) || - StringUtils.isNullOrWhitespaceOnly(hostName) || - -1 == hostPort) { - throw new ValidationException(validationExceptionMessage); - } - - hostList.add(new Host(hostName, hostPort, protocol)); - } catch (MalformedURLException e) { - throw new ValidationException(validationExceptionMessage, e); - } - } - return hostList; - } + public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; + public static final String CONNECTOR_VERSION_VALUE_6 = "6"; + public static final String CONNECTOR_VERSION_VALUE_7 = "7"; + public static final String CONNECTOR_HOSTS = "connector.hosts"; + public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; + public static final String CONNECTOR_HOSTS_PORT = "port"; + public static final String CONNECTOR_HOSTS_PROTOCOL = "protocol"; + public static final String CONNECTOR_INDEX = "connector.index"; + public static final String CONNECTOR_DOCUMENT_TYPE = "connector.document-type"; + public static final String CONNECTOR_KEY_DELIMITER = "connector.key-delimiter"; + public static final String CONNECTOR_KEY_NULL_LITERAL = "connector.key-null-literal"; + public static final String CONNECTOR_FAILURE_HANDLER = "connector.failure-handler"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_FAIL = "fail"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE = "ignore"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_RETRY = "retry-rejected"; + public static final String CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM = "custom"; + public static final String CONNECTOR_FAILURE_HANDLER_CLASS = "connector.failure-handler-class"; + public static final String CONNECTOR_FLUSH_ON_CHECKPOINT = "connector.flush-on-checkpoint"; + public static final String CONNECTOR_BULK_FLUSH_MAX_ACTIONS = + "connector.bulk-flush.max-actions"; + public static final String CONNECTOR_BULK_FLUSH_MAX_SIZE = "connector.bulk-flush.max-size"; + public static final String CONNECTOR_BULK_FLUSH_INTERVAL = "connector.bulk-flush.interval"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE = + "connector.bulk-flush.backoff.type"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED = "disabled"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT = "constant"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL = "exponential"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES = + "connector.bulk-flush.backoff.max-retries"; + public static final String CONNECTOR_BULK_FLUSH_BACKOFF_DELAY = + "connector.bulk-flush.backoff.delay"; + public static final String CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT = + "connector.connection-max-retry-timeout"; + public static final String CONNECTOR_CONNECTION_PATH_PREFIX = + "connector.connection-path-prefix"; + + @Override + public void validate(DescriptorProperties properties) { + super.validate(properties); + properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); + validateVersion(properties); + validateHosts(properties); + validateGeneralProperties(properties); + validateFailureHandler(properties); + validateBulkFlush(properties); + validateConnectionProperties(properties); + } + + private void validateVersion(DescriptorProperties properties) { + properties.validateEnumValues( + CONNECTOR_VERSION, + false, + Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7)); + } + + private void validateHosts(DescriptorProperties properties) { + if (properties.containsKey(CONNECTOR_HOSTS)) { + validateAndParseHostsString(properties); + } else { + final Map> hostsValidators = new HashMap<>(); + hostsValidators.put( + CONNECTOR_HOSTS_HOSTNAME, (key) -> properties.validateString(key, false, 1)); + hostsValidators.put( + CONNECTOR_HOSTS_PORT, (key) -> properties.validateInt(key, false, 0, 65535)); + hostsValidators.put( + CONNECTOR_HOSTS_PROTOCOL, (key) -> properties.validateString(key, false, 1)); + + properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); + } + } + + private void validateGeneralProperties(DescriptorProperties properties) { + properties.validateString(CONNECTOR_INDEX, false, 1); + properties.validateString(CONNECTOR_DOCUMENT_TYPE, false, 1); + properties.validateString(CONNECTOR_KEY_DELIMITER, true); + properties.validateString(CONNECTOR_KEY_NULL_LITERAL, true); + } + + private void validateFailureHandler(DescriptorProperties properties) { + final Map> failureHandlerValidators = new HashMap<>(); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_FAIL, noValidation()); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE, noValidation()); + failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_RETRY, noValidation()); + failureHandlerValidators.put( + CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM, + key -> properties.validateString(CONNECTOR_FAILURE_HANDLER_CLASS, false, 1)); + properties.validateEnum(CONNECTOR_FAILURE_HANDLER, true, failureHandlerValidators); + } + + private void validateBulkFlush(DescriptorProperties properties) { + properties.validateBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, true); + properties.validateInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, true, 1); + properties.validateMemorySize( + CONNECTOR_BULK_FLUSH_MAX_SIZE, true, 1024 * 1024); // only allow MB precision + properties.validateLong(CONNECTOR_BULK_FLUSH_INTERVAL, true, 0); + properties.validateEnumValues( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, + true, + Arrays.asList( + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED, + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT, + CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL)); + properties.validateInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, true, 1); + properties.validateLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, true, 0); + } + + private void validateConnectionProperties(DescriptorProperties properties) { + properties.validateInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, true, 1); + properties.validateString(CONNECTOR_CONNECTION_PATH_PREFIX, true); + } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+     *     connector.hosts = http://host_name:9092;http://host_name:9093
+     * 
+ */ + public static List validateAndParseHostsString( + DescriptorProperties descriptorProperties) { + final List hostList = new ArrayList<>(); + + descriptorProperties.validateString(CONNECTOR_HOSTS, false, 1); + final String hostsStr = descriptorProperties.getString(CONNECTOR_HOSTS); + + final String[] hosts = hostsStr.split(";"); + final String validationExceptionMessage = + "Properties '" + + CONNECTOR_HOSTS + + "' format should " + + "follow the format 'http://host_name:port', but is '" + + hostsStr + + "'."; + + if (hosts.length == 0) { + throw new ValidationException(validationExceptionMessage); + } + for (String host : hosts) { + try { + final URL url = new URL(host); + final String protocol = url.getProtocol(); + final String hostName = url.getHost(); + final int hostPort = url.getPort(); + + if (StringUtils.isNullOrWhitespaceOnly(protocol) + || StringUtils.isNullOrWhitespaceOnly(hostName) + || -1 == hostPort) { + throw new ValidationException(validationExceptionMessage); + } + + hostList.add(new Host(hostName, hostPort, protocol)); + } catch (MalformedURLException e) { + throw new ValidationException(validationExceptionMessage, e); + } + } + return hostList; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 7e55c922..285a21a7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -56,578 +56,672 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -/** - * Suite of tests for {@link ElasticsearchSinkBase}. - */ +/** Suite of tests for {@link ElasticsearchSinkBase}. */ public class ElasticsearchSinkBaseTest { - /** - * Verifies that the collection given to the sink is not modified. - */ - @Test - public void testCollectionArgumentNotModified() { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, "1"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, "true"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, "1"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, "CONSTANT"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, "1"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, "1"); - - new DummyElasticsearchSink<>( - Collections.unmodifiableMap(userConfig), - new SimpleSinkFunction(), - new NoOpFailureHandler()); - } - - /** Tests that any item failure in the listener callbacks is rethrown on an immediately following invoke call. */ - @Test - public void testItemFailureRethrownOnInvoke() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and its mock item failures - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); - testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request - sink.manualBulkRequestWithAllPendingRequests(); - - try { - testHarness.processElement(new StreamRecord<>("next msg")); - } catch (Exception e) { - // the invoke should have failed with the failure - Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for record")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** Tests that any item failure in the listener callbacks is rethrown on an immediately following checkpoint. */ - @Test - public void testItemFailureRethrownOnCheckpoint() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and its mock item failures - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); - testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request - sink.manualBulkRequestWithAllPendingRequests(); - - try { - testHarness.snapshot(1L, 1000L); - } catch (Exception e) { - // the snapshot should have failed with the failure - Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for record")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** - * Tests that any item failure in the listener callbacks due to flushing on an immediately following checkpoint - * is rethrown; we set a timeout because the test will not finish if the logic is broken. - */ - @Test(timeout = 5000) - public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and its mock item failures - - List mockResponsesList = new ArrayList<>(2); - mockResponsesList.add(null); // the first request in a bulk will succeed - mockResponsesList.add(new Exception("artificial failure for record")); // the second request in a bulk will fail - sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); - - testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request (1 request only, thus should succeed) - sink.manualBulkRequestWithAllPendingRequests(); - - // setup the requests to be flushed in the snapshot - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); - - CheckedThread snapshotThread = new CheckedThread() { - @Override - public void go() throws Exception { - testHarness.snapshot(1L, 1000L); - } - }; - snapshotThread.start(); - - // the snapshot should eventually be blocked before snapshot triggers flushing - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } - - // let the snapshot-triggered flush continue (2 records in the bulk, so the 2nd one should fail) - sink.continueFlush(); - - try { - snapshotThread.sync(); - } catch (Exception e) { - // the snapshot should have failed with the failure from the 2nd request - Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for record")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** Tests that any bulk failure in the listener callbacks is rethrown on an immediately following invoke call. */ - @Test - public void testBulkFailureRethrownOnInvoke() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and let the whole bulk request fail - sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); - testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request - sink.manualBulkRequestWithAllPendingRequests(); - - try { - testHarness.processElement(new StreamRecord<>("next msg")); - } catch (Exception e) { - // the invoke should have failed with the bulk request failure - Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for bulk request")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** Tests that any bulk failure in the listener callbacks is rethrown on an immediately following checkpoint. */ - @Test - public void testBulkFailureRethrownOnCheckpoint() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and let the whole bulk request fail - sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); - testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request - sink.manualBulkRequestWithAllPendingRequests(); - - try { - testHarness.snapshot(1L, 1000L); - } catch (Exception e) { - // the snapshot should have failed with the bulk request failure - Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for bulk request")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** - * Tests that any bulk failure in the listener callbacks due to flushing on an immediately following checkpoint - * is rethrown; we set a timeout because the test will not finish if the logic is broken. - */ - @Test(timeout = 5000) - public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new NoOpFailureHandler()); - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and let bulk request succeed - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); - testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // manually execute the next bulk request - sink.manualBulkRequestWithAllPendingRequests(); - - // setup the requests to be flushed in the snapshot - testHarness.processElement(new StreamRecord<>("msg-2")); - testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); - - CheckedThread snapshotThread = new CheckedThread() { - @Override - public void go() throws Exception { - testHarness.snapshot(1L, 1000L); - } - }; - snapshotThread.start(); - - // the snapshot should eventually be blocked before snapshot triggers flushing - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } - - // for the snapshot-triggered flush, we let the bulk request fail completely - sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); - - // let the snapshot-triggered flush continue (bulk request should fail completely) - sink.continueFlush(); - - try { - snapshotThread.sync(); - } catch (Exception e) { - // the snapshot should have failed with the bulk request failure - Assert.assertTrue(e.getCause().getCause().getMessage().contains("artificial failure for bulk request")); - - // test succeeded - return; - } - - Assert.fail(); - } - - /** - * Tests that the sink correctly waits for pending requests (including re-added requests) on checkpoints; - * we set a timeout because the test will not finish if the logic is broken. - */ - @Test(timeout = 5000) - public void testAtLeastOnceSink() throws Throwable { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), - new SimpleSinkFunction(), - new DummyRetryFailureHandler()); // use a failure handler that simply re-adds requests - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and its mock item failures; - // it contains 1 request, which will fail and re-added to the next bulk request - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); - testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - CheckedThread snapshotThread = new CheckedThread() { - @Override - public void go() throws Exception { - testHarness.snapshot(1L, 1000L); - } - }; - snapshotThread.start(); - - // the snapshot should eventually be blocked before snapshot triggers flushing - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } - - sink.continueFlush(); - - // since the previous flush should have resulted in a request re-add from the failure handler, - // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd flush - while (snapshotThread.getState() != Thread.State.WAITING) { - Thread.sleep(10); - } - - // current number of pending request should be 1 due to the re-add - Assert.assertEquals(1, sink.getNumPendingRequests()); - - // this time, let the bulk request succeed, so no-more requests are re-added - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); - - sink.continueFlush(); - - // the snapshot should finish with no exceptions - snapshotThread.sync(); - - testHarness.close(); - } - - /** - * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is disabled, - * the snapshot method does indeed finishes without waiting for pending requests; - * we set a timeout because the test will not finish if the logic is broken. - */ - @Test(timeout = 5000) - public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception { - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap(), new SimpleSinkFunction(), new DummyRetryFailureHandler()); - sink.disableFlushOnCheckpoint(); // disable flushing - - final OneInputStreamOperatorTestHarness testHarness = - new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); - - testHarness.open(); - - // setup the next bulk request, and let bulk request succeed - sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); - testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); - - // the snapshot should not block even though we haven't flushed the bulk request - testHarness.snapshot(1L, 1000L); - - testHarness.close(); - } - - @Test - public void testOpenAndCloseInSinkFunction() throws Exception { - SimpleClosableSinkFunction sinkFunction = new SimpleClosableSinkFunction<>(); - final DummyElasticsearchSink sink = new DummyElasticsearchSink<>( - new HashMap<>(), sinkFunction, new DummyRetryFailureHandler()); - - sink.open(mock(Configuration.class)); - sink.close(); - - Assert.assertTrue(sinkFunction.openCalled); - Assert.assertTrue(sinkFunction.closeCalled); - } - - private static class DummyElasticsearchSink extends ElasticsearchSinkBase { - - private static final long serialVersionUID = 5051907841570096991L; - - private transient BulkProcessor mockBulkProcessor; - private transient BulkRequest nextBulkRequest = new BulkRequest(); - private transient MultiShotLatch flushLatch = new MultiShotLatch(); - - private List mockItemFailuresList; - private Throwable nextBulkFailure; - - public DummyElasticsearchSink( - Map userConfig, - ElasticsearchSinkFunction sinkFunction, - ActionRequestFailureHandler failureHandler) { - super(new DummyElasticsearchApiCallBridge(), userConfig, sinkFunction, failureHandler); - } - - /** - * This method is used to mimic a scheduled bulk request; we need to do this - * manually because we are mocking the BulkProcessor. - */ - public void manualBulkRequestWithAllPendingRequests() { - flushLatch.trigger(); // let the flush - mockBulkProcessor.flush(); - } - - /** - * On non-manual flushes, i.e. when flush is called in the snapshot method implementation, - * usages need to explicitly call this to allow the flush to continue. This is useful - * to make sure that specific requests get added to the next bulk request for flushing. - */ - public void continueFlush() { - flushLatch.trigger(); - } - - /** - * Set the list of mock failures to use for the next bulk of item responses. A {@code null} - * means that the response is successful, failed otherwise. - * - *

The list is used with corresponding order to the requests in the bulk, i.e. the first - * request uses the response at index 0, the second requests uses the response at index 1, etc. - */ - public void setMockItemFailuresListForNextBulkItemResponses(List mockItemFailuresList) { - this.mockItemFailuresList = mockItemFailuresList; - } - - /** - * Let the next bulk request fail completely with the provided throwable. - * If this is set, the failures list provided with setMockItemFailuresListForNextBulkItemResponses is not respected. - */ - public void setFailNextBulkRequestCompletely(Throwable failure) { - this.nextBulkFailure = failure; - } - - public BulkProcessor getMockBulkProcessor() { - return mockBulkProcessor; - } - - /** - * Override the bulk processor build process to provide a mock implementation, - * but reuse the listener implementation in our mock to test that the listener logic - * works correctly with request flushing logic. - */ - @Override - protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { - this.mockBulkProcessor = mock(BulkProcessor.class); - - when(mockBulkProcessor.add(any(IndexRequest.class))).thenAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) throws Throwable { - // intercept the request and add it to our mock bulk request - nextBulkRequest.add((IndexRequest) invocationOnMock.getArgument(0)); - - return null; - } - }); - - doAnswer(new Answer() { - @Override - public Object answer(InvocationOnMock invocationOnMock) throws Throwable { - while (nextBulkRequest.numberOfActions() > 0) { - // wait until we are allowed to continue with the flushing - flushLatch.await(); - - // create a copy of the accumulated mock requests, so that - // re-added requests from the failure handler are included in the next bulk - BulkRequest currentBulkRequest = nextBulkRequest; - nextBulkRequest = new BulkRequest(); - - listener.beforeBulk(123L, currentBulkRequest); - - if (nextBulkFailure == null) { - BulkItemResponse[] mockResponses = new BulkItemResponse[currentBulkRequest.requests().size()]; - for (int i = 0; i < currentBulkRequest.requests().size(); i++) { - Exception mockItemFailure = mockItemFailuresList.get(i); - - if (mockItemFailure == null) { - // the mock response for the item is success - mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, mock(DocWriteResponse.class)); - } else { - // the mock response for the item is failure - mockResponses[i] = new BulkItemResponse(i, DocWriteRequest.OpType.INDEX, new BulkItemResponse.Failure("index", "type", "id", mockItemFailure)); - } - } - - listener.afterBulk(123L, currentBulkRequest, new BulkResponse(mockResponses, 1000L)); - } else { - listener.afterBulk(123L, currentBulkRequest, nextBulkFailure); - } - } - - return null; - } - }).when(mockBulkProcessor).flush(); - - return mockBulkProcessor; - } - } - - private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { - - private static final long serialVersionUID = -4272760730959041699L; - - @Override - public Client createClient(Map clientConfig) { - return mock(Client.class); - } - - @Override - public BulkProcessor.Builder createBulkProcessorBuilder(Client client, BulkProcessor.Listener listener) { - return null; - } - - @Nullable - @Override - public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { - if (bulkItemResponse.isFailed()) { - return new Exception(bulkItemResponse.getFailure().getMessage()); - } else { - return null; - } - } - - @Override - public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { - // no need for this in the test cases here - } - - @Override - public void verifyClientConnection(Client client) { - // no need for this in the test cases here - } - } - - private static class SimpleSinkFunction implements ElasticsearchSinkFunction { - - private static final long serialVersionUID = -176739293659135148L; - - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - Map json = new HashMap<>(); - json.put("data", element); - - indexer.add( - Requests.indexRequest() - .index("index") - .type("type") - .id("id") - .source(json) - ); - } - } - - private static class SimpleClosableSinkFunction implements ElasticsearchSinkFunction { - - private static final long serialVersionUID = 1872065917794006848L; - - private boolean openCalled; - private boolean closeCalled; - - @Override - public void open() { - openCalled = true; - } - - @Override - public void close() { - closeCalled = true; - } - - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {} - } - - private static class DummyRetryFailureHandler implements ActionRequestFailureHandler { - - private static final long serialVersionUID = 5400023700099200745L; - - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) throws Throwable { - indexer.add(action); - } - } + /** Verifies that the collection given to the sink is not modified. */ + @Test + public void testCollectionArgumentNotModified() { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, "true"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, "CONSTANT"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, "1"); + + new DummyElasticsearchSink<>( + Collections.unmodifiableMap(userConfig), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + } + + /** + * Tests that any item failure in the listener callbacks is rethrown on an immediately following + * invoke call. + */ + @Test + public void testItemFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the failure + Assert.assertTrue(e.getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any item failure in the listener callbacks is rethrown on an immediately following + * checkpoint. + */ + @Test + public void testItemFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the failure + Assert.assertTrue( + e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any item failure in the listener callbacks due to flushing on an immediately + * following checkpoint is rethrown; we set a timeout because the test will not finish if the + * logic is broken. + */ + @Test(timeout = 5000) + public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures + + List mockResponsesList = new ArrayList<>(2); + mockResponsesList.add(null); // the first request in a bulk will succeed + mockResponsesList.add( + new Exception( + "artificial failure for record")); // the second request in a bulk will fail + sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); + + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request (1 request only, thus should succeed) + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // let the snapshot-triggered flush continue (2 records in the bulk, so the 2nd one should + // fail) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the failure from the 2nd request + Assert.assertTrue( + e.getCause().getCause().getMessage().contains("artificial failure for record")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following + * invoke call. + */ + @Test + public void testBulkFailureRethrownOnInvoke() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.processElement(new StreamRecord<>("next msg")); + } catch (Exception e) { + // the invoke should have failed with the bulk request failure + Assert.assertTrue( + e.getCause().getMessage().contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks is rethrown on an immediately following + * checkpoint. + */ + @Test + public void testBulkFailureRethrownOnCheckpoint() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let the whole bulk request fail + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + try { + testHarness.snapshot(1L, 1000L); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue( + e.getCause() + .getCause() + .getMessage() + .contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that any bulk failure in the listener callbacks due to flushing on an immediately + * following checkpoint is rethrown; we set a timeout because the test will not finish if the + * logic is broken. + */ + @Test(timeout = 5000) + public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new NoOpFailureHandler()); + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList((Exception) null)); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // manually execute the next bulk request + sink.manualBulkRequestWithAllPendingRequests(); + + // setup the requests to be flushed in the snapshot + testHarness.processElement(new StreamRecord<>("msg-2")); + testHarness.processElement(new StreamRecord<>("msg-3")); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // for the snapshot-triggered flush, we let the bulk request fail completely + sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); + + // let the snapshot-triggered flush continue (bulk request should fail completely) + sink.continueFlush(); + + try { + snapshotThread.sync(); + } catch (Exception e) { + // the snapshot should have failed with the bulk request failure + Assert.assertTrue( + e.getCause() + .getCause() + .getMessage() + .contains("artificial failure for bulk request")); + + // test succeeded + return; + } + + Assert.fail(); + } + + /** + * Tests that the sink correctly waits for pending requests (including re-added requests) on + * checkpoints; we set a timeout because the test will not finish if the logic is broken. + */ + @Test(timeout = 5000) + public void testAtLeastOnceSink() throws Throwable { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new DummyRetryFailureHandler()); // use a failure handler that simply + // re-adds requests + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and its mock item failures; + // it contains 1 request, which will fail and re-added to the next bulk request + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + CheckedThread snapshotThread = + new CheckedThread() { + @Override + public void go() throws Exception { + testHarness.snapshot(1L, 1000L); + } + }; + snapshotThread.start(); + + // the snapshot should eventually be blocked before snapshot triggers flushing + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + sink.continueFlush(); + + // since the previous flush should have resulted in a request re-add from the failure + // handler, + // we should have flushed again, and eventually be blocked before snapshot triggers the 2nd + // flush + while (snapshotThread.getState() != Thread.State.WAITING) { + Thread.sleep(10); + } + + // current number of pending request should be 1 due to the re-add + Assert.assertEquals(1, sink.getNumPendingRequests()); + + // this time, let the bulk request succeed, so no-more requests are re-added + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList((Exception) null)); + + sink.continueFlush(); + + // the snapshot should finish with no exceptions + snapshotThread.sync(); + + testHarness.close(); + } + + /** + * This test is meant to assure that testAtLeastOnceSink is valid by testing that if flushing is + * disabled, the snapshot method does indeed finishes without waiting for pending requests; we + * set a timeout because the test will not finish if the logic is broken. + */ + @Test(timeout = 5000) + public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Exception { + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap(), + new SimpleSinkFunction(), + new DummyRetryFailureHandler()); + sink.disableFlushOnCheckpoint(); // disable flushing + + final OneInputStreamOperatorTestHarness testHarness = + new OneInputStreamOperatorTestHarness<>(new StreamSink<>(sink)); + + testHarness.open(); + + // setup the next bulk request, and let bulk request succeed + sink.setMockItemFailuresListForNextBulkItemResponses( + Collections.singletonList(new Exception("artificial failure for record"))); + testHarness.processElement(new StreamRecord<>("msg-1")); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); + + // the snapshot should not block even though we haven't flushed the bulk request + testHarness.snapshot(1L, 1000L); + + testHarness.close(); + } + + @Test + public void testOpenAndCloseInSinkFunction() throws Exception { + SimpleClosableSinkFunction sinkFunction = new SimpleClosableSinkFunction<>(); + final DummyElasticsearchSink sink = + new DummyElasticsearchSink<>( + new HashMap<>(), sinkFunction, new DummyRetryFailureHandler()); + + sink.open(mock(Configuration.class)); + sink.close(); + + Assert.assertTrue(sinkFunction.openCalled); + Assert.assertTrue(sinkFunction.closeCalled); + } + + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 5051907841570096991L; + + private transient BulkProcessor mockBulkProcessor; + private transient BulkRequest nextBulkRequest = new BulkRequest(); + private transient MultiShotLatch flushLatch = new MultiShotLatch(); + + private List mockItemFailuresList; + private Throwable nextBulkFailure; + + public DummyElasticsearchSink( + Map userConfig, + ElasticsearchSinkFunction sinkFunction, + ActionRequestFailureHandler failureHandler) { + super(new DummyElasticsearchApiCallBridge(), userConfig, sinkFunction, failureHandler); + } + + /** + * This method is used to mimic a scheduled bulk request; we need to do this manually + * because we are mocking the BulkProcessor. + */ + public void manualBulkRequestWithAllPendingRequests() { + flushLatch.trigger(); // let the flush + mockBulkProcessor.flush(); + } + + /** + * On non-manual flushes, i.e. when flush is called in the snapshot method implementation, + * usages need to explicitly call this to allow the flush to continue. This is useful to + * make sure that specific requests get added to the next bulk request for flushing. + */ + public void continueFlush() { + flushLatch.trigger(); + } + + /** + * Set the list of mock failures to use for the next bulk of item responses. A {@code null} + * means that the response is successful, failed otherwise. + * + *

The list is used with corresponding order to the requests in the bulk, i.e. the first + * request uses the response at index 0, the second requests uses the response at index 1, + * etc. + */ + public void setMockItemFailuresListForNextBulkItemResponses( + List mockItemFailuresList) { + this.mockItemFailuresList = mockItemFailuresList; + } + + /** + * Let the next bulk request fail completely with the provided throwable. If this is set, + * the failures list provided with setMockItemFailuresListForNextBulkItemResponses is not + * respected. + */ + public void setFailNextBulkRequestCompletely(Throwable failure) { + this.nextBulkFailure = failure; + } + + public BulkProcessor getMockBulkProcessor() { + return mockBulkProcessor; + } + + /** + * Override the bulk processor build process to provide a mock implementation, but reuse the + * listener implementation in our mock to test that the listener logic works correctly with + * request flushing logic. + */ + @Override + protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { + this.mockBulkProcessor = mock(BulkProcessor.class); + + when(mockBulkProcessor.add(any(IndexRequest.class))) + .thenAnswer( + new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) + throws Throwable { + // intercept the request and add it to our mock bulk request + nextBulkRequest.add( + (IndexRequest) invocationOnMock.getArgument(0)); + + return null; + } + }); + + doAnswer( + new Answer() { + @Override + public Object answer(InvocationOnMock invocationOnMock) + throws Throwable { + while (nextBulkRequest.numberOfActions() > 0) { + // wait until we are allowed to continue with the flushing + flushLatch.await(); + + // create a copy of the accumulated mock requests, so that + // re-added requests from the failure handler are included + // in the next bulk + BulkRequest currentBulkRequest = nextBulkRequest; + nextBulkRequest = new BulkRequest(); + + listener.beforeBulk(123L, currentBulkRequest); + + if (nextBulkFailure == null) { + BulkItemResponse[] mockResponses = + new BulkItemResponse + [currentBulkRequest.requests().size()]; + for (int i = 0; + i < currentBulkRequest.requests().size(); + i++) { + Exception mockItemFailure = + mockItemFailuresList.get(i); + + if (mockItemFailure == null) { + // the mock response for the item is success + mockResponses[i] = + new BulkItemResponse( + i, + DocWriteRequest.OpType.INDEX, + mock(DocWriteResponse.class)); + } else { + // the mock response for the item is failure + mockResponses[i] = + new BulkItemResponse( + i, + DocWriteRequest.OpType.INDEX, + new BulkItemResponse.Failure( + "index", + "type", + "id", + mockItemFailure)); + } + } + + listener.afterBulk( + 123L, + currentBulkRequest, + new BulkResponse(mockResponses, 1000L)); + } else { + listener.afterBulk( + 123L, currentBulkRequest, nextBulkFailure); + } + } + + return null; + } + }) + .when(mockBulkProcessor) + .flush(); + + return mockBulkProcessor; + } + } + + private static class DummyElasticsearchApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -4272760730959041699L; + + @Override + public Client createClient(Map clientConfig) { + return mock(Client.class); + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + Client client, BulkProcessor.Listener listener) { + return null; + } + + @Nullable + @Override + public Throwable extractFailureCauseFromBulkItemResponse( + BulkItemResponse bulkItemResponse) { + if (bulkItemResponse.isFailed()) { + return new Exception(bulkItemResponse.getFailure().getMessage()); + } else { + return null; + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + // no need for this in the test cases here + } + + @Override + public void verifyClientConnection(Client client) { + // no need for this in the test cases here + } + } + + private static class SimpleSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = -176739293659135148L; + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + Map json = new HashMap<>(); + json.put("data", element); + + indexer.add(Requests.indexRequest().index("index").type("type").id("id").source(json)); + } + } + + private static class SimpleClosableSinkFunction + implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1872065917794006848L; + + private boolean openCalled; + private boolean closeCalled; + + @Override + public void open() { + openCalled = true; + } + + @Override + public void close() { + closeCalled = true; + } + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {} + } + + private static class DummyRetryFailureHandler implements ActionRequestFailureHandler { + + private static final long serialVersionUID = 5400023700099200745L; + + @Override + public void onFailure( + ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) + throws Throwable { + indexer.add(action); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 034161ec..1d250b7e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -35,164 +35,171 @@ import static org.junit.Assert.fail; /** - * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} implementations. + * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} + * implementations. * * @param Elasticsearch client type * @param The address type to use */ -public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { - - // It's not good that we're using a Client here instead of a Rest Client but we need this - // for compatibility with ES 5.3.x. As soon as we drop that we can use RestClient here. - protected abstract Client getClient(); - - protected abstract String getClusterName(); - - /** - * Tests that the Elasticsearch sink works properly with json. - */ - public void runElasticsearchSinkTest() throws Exception { - runElasticSearchSinkTest("elasticsearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction); - } - - /** - * Tests that the Elasticsearch sink works properly with cbor. - */ - public void runElasticsearchSinkCborTest() throws Exception { - runElasticSearchSinkTest("elasticsearch-sink-test-cbor-index", SourceSinkDataTestKit::getCborSinkFunction); - } - - /** - * Tests that the Elasticsearch sink works properly with smile. - */ - public void runElasticsearchSinkSmileTest() throws Exception { - runElasticSearchSinkTest("elasticsearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction); - } - - /** - * Tests that the Elasticsearch sink works properly with yaml. - */ - public void runElasticsearchSinkYamlTest() throws Exception { - runElasticSearchSinkTest("elasticsearch-sink-test-yaml-index", SourceSinkDataTestKit::getYamlSinkFunction); - } - - private void runElasticSearchSinkTest(String index, Function>> functionFactory) throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - source.addSink(createElasticsearchSinkForEmbeddedNode( - 1, - getClusterName(), - functionFactory.apply(index))); - - env.execute("Elasticsearch Sink Test"); - - // verify the results - Client client = getClient(); - - SourceSinkDataTestKit.verifyProducedSinkData(client, index); - - client.close(); - } - - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code null}. - */ - public void runNullAddressesTest() { - try { - createElasticsearchSink( - 1, - getClusterName(), - null, - SourceSinkDataTestKit.getJsonSinkFunction("test")); - } catch (IllegalArgumentException | NullPointerException expectedException) { - // test passes - return; - } - - fail(); - } - - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. - */ - public void runEmptyAddressesTest() { - try { - createElasticsearchSink( - 1, - getClusterName(), - Collections.emptyList(), - SourceSinkDataTestKit.getJsonSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { - // test passes - return; - } - - fail(); - } - - /** - * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. - */ - public void runInvalidElasticsearchClusterTest() throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - source.addSink(createElasticsearchSinkForNode( - 1, - "invalid-cluster-name", - SourceSinkDataTestKit.getJsonSinkFunction("test"), - "123.123.123.123")); // incorrect ip address - - try { - env.execute("Elasticsearch Sink Test"); - } catch (JobExecutionException expectedException) { - // every ES version throws a different exception in case of timeouts, so don't bother asserting on the exception - // test passes - return; - } - - fail(); - } - - /** - * Utility method to create a user config map. - */ - protected Map createUserConfig(int bulkFlushMaxActions, String clusterName) { - Map userConfig = new HashMap<>(); - userConfig.put("cluster.name", clusterName); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(bulkFlushMaxActions)); - userConfig.put("transport.tcp.connect_timeout", "5s"); - - return userConfig; - } - - /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ - protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( - int bulkFlushMaxActions, - String clusterName, - List addresses, - ElasticsearchSinkFunction> elasticsearchSinkFunction); - - /** - * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch node. - * - *

This case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(int, String, List, ElasticsearchSinkFunction)} - * because the Elasticsearch Java API to do so is incompatible across different versions. - */ - protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; - - /** - * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. - */ - protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction, - String ipAddress) throws Exception; +public abstract class ElasticsearchSinkTestBase + extends AbstractTestBase { + + // It's not good that we're using a Client here instead of a Rest Client but we need this + // for compatibility with ES 5.3.x. As soon as we drop that we can use RestClient here. + protected abstract Client getClient(); + + protected abstract String getClusterName(); + + /** Tests that the Elasticsearch sink works properly with json. */ + public void runElasticsearchSinkTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-json-index", SourceSinkDataTestKit::getJsonSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with cbor. */ + public void runElasticsearchSinkCborTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-cbor-index", SourceSinkDataTestKit::getCborSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with smile. */ + public void runElasticsearchSinkSmileTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-smile-index", SourceSinkDataTestKit::getSmileSinkFunction); + } + + /** Tests that the Elasticsearch sink works properly with yaml. */ + public void runElasticsearchSinkYamlTest() throws Exception { + runElasticSearchSinkTest( + "elasticsearch-sink-test-yaml-index", SourceSinkDataTestKit::getYamlSinkFunction); + } + + private void runElasticSearchSinkTest( + String index, + Function>> functionFactory) + throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createElasticsearchSinkForEmbeddedNode( + 1, getClusterName(), functionFactory.apply(index))); + + env.execute("Elasticsearch Sink Test"); + + // verify the results + Client client = getClient(); + + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code + * null}. + */ + public void runNullAddressesTest() { + try { + createElasticsearchSink( + 1, getClusterName(), null, SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException | NullPointerException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. + */ + public void runEmptyAddressesTest() { + try { + createElasticsearchSink( + 1, + getClusterName(), + Collections.emptyList(), + SourceSinkDataTestKit.getJsonSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** Tests whether the Elasticsearch sink fails when there is no cluster to connect to. */ + public void runInvalidElasticsearchClusterTest() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = + env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + source.addSink( + createElasticsearchSinkForNode( + 1, + "invalid-cluster-name", + SourceSinkDataTestKit.getJsonSinkFunction("test"), + "123.123.123.123")); // incorrect ip address + + try { + env.execute("Elasticsearch Sink Test"); + } catch (JobExecutionException expectedException) { + // every ES version throws a different exception in case of timeouts, so don't bother + // asserting on the exception + // test passes + return; + } + + fail(); + } + + /** Utility method to create a user config map. */ + protected Map createUserConfig(int bulkFlushMaxActions, String clusterName) { + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", clusterName); + userConfig.put( + ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, + String.valueOf(bulkFlushMaxActions)); + userConfig.put("transport.tcp.connect_timeout", "5s"); + + return userConfig; + } + + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ + protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List addresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction); + + /** + * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch + * node. + * + *

This case is singled out from {@link + * ElasticsearchSinkTestBase#createElasticsearchSink(int, String, List, + * ElasticsearchSinkFunction)} because the Elasticsearch Java API to do so is incompatible + * across different versions. + */ + protected abstract ElasticsearchSinkBase, C> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) + throws Exception; + + /** + * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. + */ + protected abstract ElasticsearchSinkBase, C> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) + throws Exception; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index ced65a00..03e5b241 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -47,185 +47,191 @@ import static org.junit.Assert.assertEquals; -/** - * Version-agnostic test base for {@link ElasticsearchUpsertTableSinkFactoryBase}. - */ +/** Version-agnostic test base for {@link ElasticsearchUpsertTableSinkFactoryBase}. */ public abstract class ElasticsearchUpsertTableSinkFactoryTestBase extends TestLogger { - protected static final String HOSTNAME = "host1"; - protected static final int PORT = 1234; - protected static final String SCHEMA = "https"; - protected static final String INDEX = "MyIndex"; - protected static final String DOC_TYPE = "MyType"; - protected static final String KEY_DELIMITER = "#"; - protected static final String KEY_NULL_LITERAL = ""; - - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - @Test - public void testTableSink() { - // prepare parameters for Elasticsearch table sink - - final TableSchema schema = createTestSchema(); - - final ElasticsearchUpsertTableSinkBase expectedSink = getExpectedTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions(), - IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); - - // construct table sink using descriptors and table sink factory - final Map elasticSearchProperties = createElasticSearchProperties(); - final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, elasticSearchProperties) - .createStreamTableSink(elasticSearchProperties); - - assertEquals(expectedSink, actualSink); - } - - @Test - public void testTableSinkWithLegacyProperties() { - // prepare parameters for Elasticsearch table sink - final TableSchema schema = createTestSchema(); - - final ElasticsearchUpsertTableSinkBase expectedSink = getExpectedTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions(), - IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); - - // construct table sink using descriptors and table sink factory - final Map elasticSearchProperties = createElasticSearchProperties(); - - final Map legacyPropertiesMap = new HashMap<>(); - legacyPropertiesMap.putAll(elasticSearchProperties); - // use legacy properties - legacyPropertiesMap.remove("connector.hosts"); - - legacyPropertiesMap.put("connector.hosts.0.hostname", "host1"); - legacyPropertiesMap.put("connector.hosts.0.port", "1234"); - legacyPropertiesMap.put("connector.hosts.0.protocol", "https"); - - final TableSink actualSink = TableFactoryService.find(StreamTableSinkFactory.class, legacyPropertiesMap) - .createStreamTableSink(legacyPropertiesMap); - - assertEquals(expectedSink, actualSink); - } - - protected TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - protected Map createTestSinkOptions() { - final Map sinkOptions = new HashMap<>(); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_ENABLED, "true"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_TYPE, "EXPONENTIAL"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_DELAY, "123"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_RETRIES, "3"); - sinkOptions.put(SinkOption.BULK_FLUSH_INTERVAL, "100"); - sinkOptions.put(SinkOption.BULK_FLUSH_MAX_ACTIONS, "1000"); - sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1 mb"); - sinkOptions.put(SinkOption.REST_MAX_RETRY_TIMEOUT, "100"); - sinkOptions.put(SinkOption.REST_PATH_PREFIX, "/myapp"); - return sinkOptions; - } - - protected Map createElasticSearchProperties() { - return new TestTableDescriptor( - new Elasticsearch() - .version(getElasticsearchVersion()) - .host(HOSTNAME, PORT, SCHEMA) - .index(INDEX) - .documentType(DOC_TYPE) - .keyDelimiter(KEY_DELIMITER) - .keyNullLiteral(KEY_NULL_LITERAL) - .bulkFlushBackoffExponential() - .bulkFlushBackoffDelay(123L) - .bulkFlushBackoffMaxRetries(3) - .bulkFlushInterval(100L) - .bulkFlushMaxActions(1000) - .bulkFlushMaxSize("1 MB") - .failureHandlerCustom(DummyFailureHandler.class) - .connectionMaxRetryTimeout(100) - .connectionPathPrefix("/myapp")) - .withFormat( - new Json() - .deriveSchema()) - .withSchema( - new Schema() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3))) - .inUpsertMode() - .toProperties(); - } - - // -------------------------------------------------------------------------------------------- - // For version-specific tests - // -------------------------------------------------------------------------------------------- - - protected abstract String getElasticsearchVersion(); - - protected abstract ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator); - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** - * Custom failure handler for testing. - */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return this == o || o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } + protected static final String HOSTNAME = "host1"; + protected static final int PORT = 1234; + protected static final String SCHEMA = "https"; + protected static final String INDEX = "MyIndex"; + protected static final String DOC_TYPE = "MyType"; + protected static final String KEY_DELIMITER = "#"; + protected static final String KEY_NULL_LITERAL = ""; + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + @Test + public void testTableSink() { + // prepare parameters for Elasticsearch table sink + + final TableSchema schema = createTestSchema(); + + final ElasticsearchUpsertTableSinkBase expectedSink = + getExpectedTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions(), + IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); + + // construct table sink using descriptors and table sink factory + final Map elasticSearchProperties = createElasticSearchProperties(); + final TableSink actualSink = + TableFactoryService.find(StreamTableSinkFactory.class, elasticSearchProperties) + .createStreamTableSink(elasticSearchProperties); + + assertEquals(expectedSink, actualSink); + } + + @Test + public void testTableSinkWithLegacyProperties() { + // prepare parameters for Elasticsearch table sink + final TableSchema schema = createTestSchema(); + + final ElasticsearchUpsertTableSinkBase expectedSink = + getExpectedTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions(), + IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); + + // construct table sink using descriptors and table sink factory + final Map elasticSearchProperties = createElasticSearchProperties(); + + final Map legacyPropertiesMap = new HashMap<>(); + legacyPropertiesMap.putAll(elasticSearchProperties); + // use legacy properties + legacyPropertiesMap.remove("connector.hosts"); + + legacyPropertiesMap.put("connector.hosts.0.hostname", "host1"); + legacyPropertiesMap.put("connector.hosts.0.port", "1234"); + legacyPropertiesMap.put("connector.hosts.0.protocol", "https"); + + final TableSink actualSink = + TableFactoryService.find(StreamTableSinkFactory.class, legacyPropertiesMap) + .createStreamTableSink(legacyPropertiesMap); + + assertEquals(expectedSink, actualSink); + } + + protected TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + protected Map createTestSinkOptions() { + final Map sinkOptions = new HashMap<>(); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_ENABLED, "true"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_TYPE, "EXPONENTIAL"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_DELAY, "123"); + sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_RETRIES, "3"); + sinkOptions.put(SinkOption.BULK_FLUSH_INTERVAL, "100"); + sinkOptions.put(SinkOption.BULK_FLUSH_MAX_ACTIONS, "1000"); + sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1 mb"); + sinkOptions.put(SinkOption.REST_MAX_RETRY_TIMEOUT, "100"); + sinkOptions.put(SinkOption.REST_PATH_PREFIX, "/myapp"); + return sinkOptions; + } + + protected Map createElasticSearchProperties() { + return new TestTableDescriptor( + new Elasticsearch() + .version(getElasticsearchVersion()) + .host(HOSTNAME, PORT, SCHEMA) + .index(INDEX) + .documentType(DOC_TYPE) + .keyDelimiter(KEY_DELIMITER) + .keyNullLiteral(KEY_NULL_LITERAL) + .bulkFlushBackoffExponential() + .bulkFlushBackoffDelay(123L) + .bulkFlushBackoffMaxRetries(3) + .bulkFlushInterval(100L) + .bulkFlushMaxActions(1000) + .bulkFlushMaxSize("1 MB") + .failureHandlerCustom(DummyFailureHandler.class) + .connectionMaxRetryTimeout(100) + .connectionPathPrefix("/myapp")) + .withFormat(new Json().deriveSchema()) + .withSchema( + new Schema() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3))) + .inUpsertMode() + .toProperties(); + } + + // -------------------------------------------------------------------------------------------- + // For version-specific tests + // -------------------------------------------------------------------------------------------- + + protected abstract String getElasticsearchVersion(); + + protected abstract ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + IndexGenerator indexGenerator); + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return this == o || o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java index fd14ba36..738ddd60 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -22,34 +22,33 @@ import java.io.File; /** - * The {@link EmbeddedElasticsearchNodeEnvironment} is used in integration tests to manage Elasticsearch embedded nodes. + * The {@link EmbeddedElasticsearchNodeEnvironment} is used in integration tests to manage + * Elasticsearch embedded nodes. * - *

NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific implementations - * for the tests, concrete implementations must be named {@code EmbeddedElasticsearchNodeEnvironmentImpl}. It must - * also be located under the same package. The intentional package-private accessibility of this interface - * enforces that. + *

NOTE: In order for {@link ElasticsearchSinkTestBase} to dynamically load version-specific + * implementations for the tests, concrete implementations must be named {@code + * EmbeddedElasticsearchNodeEnvironmentImpl}. It must also be located under the same package. The + * intentional package-private accessibility of this interface enforces that. */ public interface EmbeddedElasticsearchNodeEnvironment { - /** - * Start an embedded Elasticsearch node instance. - * Calling this method multiple times consecutively should not restart the embedded node. - * - * @param tmpDataFolder The temporary data folder for the embedded node to use. - * @param clusterName The name of the cluster that the embedded node should be configured with. - */ - void start(File tmpDataFolder, String clusterName) throws Exception; + /** + * Start an embedded Elasticsearch node instance. Calling this method multiple times + * consecutively should not restart the embedded node. + * + * @param tmpDataFolder The temporary data folder for the embedded node to use. + * @param clusterName The name of the cluster that the embedded node should be configured with. + */ + void start(File tmpDataFolder, String clusterName) throws Exception; - /** - * Close the embedded node, if previously started. - */ - void close() throws Exception; + /** Close the embedded node, if previously started. */ + void close() throws Exception; - /** - * Returns a client to communicate with the embedded node. - * - * @return Client to communicate with the embedded node. - * Returns {@code null} if the embedded node wasn't started or is closed. - */ - Client getClient(); + /** + * Returns a client to communicate with the embedded node. + * + * @return Client to communicate with the embedded node. Returns {@code null} if the embedded + * node wasn't started or is closed. + */ + Client getClient(); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java index 89a67bbd..522788f5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java @@ -38,209 +38,223 @@ import java.util.ArrayList; import java.util.List; -/** - * Suite tests for {@link IndexGenerator}. - */ +/** Suite tests for {@link IndexGenerator}. */ public class IndexGeneratorTest { - private TableSchema schema; - private List rows; - - @Before - public void prepareData() { - String[] fieldNames = new String[]{ - "id", - "item", - "log_ts", - "log_date", - "order_timestamp", - "log_time", - "local_datetime", - "local_date", - "local_time", - "note", - "status"}; - DataType[] dataTypes = new DataType[]{ - DataTypes.INT(), - DataTypes.STRING(), - DataTypes.BIGINT(), - DataTypes.DATE().bridgedTo(java.sql.Date.class), - DataTypes.TIMESTAMP().bridgedTo(java.sql.Timestamp.class), - DataTypes.TIME().bridgedTo(java.sql.Time.class), - DataTypes.TIMESTAMP().bridgedTo(java.time.LocalDateTime.class), - DataTypes.DATE().bridgedTo(java.time.LocalDate.class), - DataTypes.TIME().bridgedTo(java.time.LocalTime.class), - DataTypes.STRING(), - DataTypes.BOOLEAN() - }; - schema = new TableSchema.Builder().fields(fieldNames, dataTypes).build(); - - rows = new ArrayList<>(); - rows.add(Row.of( - 1, - "apple", - Timestamp.valueOf("2020-03-18 12:12:14").getTime(), - Date.valueOf("2020-03-18"), - Timestamp.valueOf("2020-03-18 12:12:14"), - Time.valueOf("12:12:14"), - LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000), - LocalDate.of(2020, 3, 18), - LocalTime.of(12, 13, 14, 2000), - "test1", - true)); - rows.add(Row.of( - 2, - "peanut", - Timestamp.valueOf("2020-03-19 12:22:14").getTime(), - Date.valueOf("2020-03-19"), - Timestamp.valueOf("2020-03-19 12:22:21"), - Time.valueOf("12:22:21"), - LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000), - LocalDate.of(2020, 3, 19), - LocalTime.of(12, 13, 14, 2000), - "test2", - false)); - } - - @Test - public void testDynamicIndexFromTimestamp() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalDateTime() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "{local_datetime|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = IndexGeneratorFactory.createIndexGenerator( - "{local_datetime|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromDate() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalDate() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromTime() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalTime() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormat() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_time|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testGeneralDynamicIndex() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "index_{item}", schema); - indexGenerator.open(); - Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testStaticIndex() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index", schema); - indexGenerator.open(); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testUnknownField() { - String expectedExceptionMsg = "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + - " please check the field name."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(e.getMessage(), expectedExceptionMsg); - } - } - - @Test - public void testUnsupportedTimeType() { - String expectedExceptionMsg = "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + - "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedMultiParametersType() { - String expectedExceptionMsg = "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + - " only support single dynamic index pattern."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{local_date}-{local_time}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testDynamicIndexUnsupportedFormat() { - String expectedExceptionMsg = "Unsupported field: HourOfDay"; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date|yyyy/MM/dd HH:mm}", schema); - } catch (UnsupportedTemporalTypeException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedIndexFieldType() { - String expectedExceptionMsg = "Unsupported type Boolean of index field, Supported types are:" + - " [LocalDateTime, Timestamp, LocalDate, Date, LocalTime, Time, String, Short, Integer, Long]"; - try { - IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); - } catch (IllegalArgumentException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + String[] fieldNames = + new String[] { + "id", + "item", + "log_ts", + "log_date", + "order_timestamp", + "log_time", + "local_datetime", + "local_date", + "local_time", + "note", + "status" + }; + DataType[] dataTypes = + new DataType[] { + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.DATE().bridgedTo(java.sql.Date.class), + DataTypes.TIMESTAMP().bridgedTo(java.sql.Timestamp.class), + DataTypes.TIME().bridgedTo(java.sql.Time.class), + DataTypes.TIMESTAMP().bridgedTo(java.time.LocalDateTime.class), + DataTypes.DATE().bridgedTo(java.time.LocalDate.class), + DataTypes.TIME().bridgedTo(java.time.LocalTime.class), + DataTypes.STRING(), + DataTypes.BOOLEAN() + }; + schema = new TableSchema.Builder().fields(fieldNames, dataTypes).build(); + + rows = new ArrayList<>(); + rows.add( + Row.of( + 1, + "apple", + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + Date.valueOf("2020-03-18"), + Timestamp.valueOf("2020-03-18 12:12:14"), + Time.valueOf("12:12:14"), + LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000), + LocalDate.of(2020, 3, 18), + LocalTime.of(12, 13, 14, 2000), + "test1", + true)); + rows.add( + Row.of( + 2, + "peanut", + Timestamp.valueOf("2020-03-19 12:22:14").getTime(), + Date.valueOf("2020-03-19"), + Timestamp.valueOf("2020-03-19 12:22:21"), + Time.valueOf("12:22:21"), + LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000), + LocalDate.of(2020, 3, 19), + LocalTime.of(12, 13, 14, 2000), + "test2", + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDateTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type Boolean of index field, Supported types are:" + + " [LocalDateTime, Timestamp, LocalDate, Date, LocalTime, Time, String, Short, Integer, Long]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java index 2f0eedd4..081864fe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java @@ -40,174 +40,176 @@ import java.util.ArrayList; import java.util.List; -/** - * Tests for {@link IndexGeneratorFactory}. - */ +/** Tests for {@link IndexGeneratorFactory}. */ public class IndexGeneratorFactoryTest extends TestLogger { - private TableSchema schema; - private List rows; - - @Before - public void prepareData() { - schema = new TableSchema.Builder() - .field("id", DataTypes.INT()) - .field("item", DataTypes.STRING()) - .field("log_ts", DataTypes.BIGINT()) - .field("log_date", DataTypes.DATE()) - .field("log_time", DataTypes.TIME()) - .field("order_timestamp", DataTypes.TIMESTAMP()) - .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("status", DataTypes.BOOLEAN()) - .build(); - - rows = new ArrayList<>(); - rows.add(GenericRowData.of( - 1, - StringData.fromString("apple"), - Timestamp.valueOf("2020-03-18 12:12:14").getTime(), - (int) LocalDate.parse("2020-03-18").toEpochDay(), - (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), - TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), - true)); - rows.add(GenericRowData.of( - 2, - StringData.fromString("peanut"), - Timestamp.valueOf("2020-03-19 12:12:14").getTime(), - (int) LocalDate.parse("2020-03-19").toEpochDay(), - (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), - TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), - false)); - } - - @Test - public void testDynamicIndexFromTimestamp() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = IndexGeneratorFactory - .createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromDate() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromTime() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "my-index-{log_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormat() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "my-index-{log_time|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "my-index-{local_timestamp|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testGeneralDynamicIndex() { - IndexGenerator indexGenerator = IndexGeneratorFactory - .createIndexGenerator( - "index_{item}", schema); - indexGenerator.open(); - Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testStaticIndex() { - IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator( - "my-index", schema); - indexGenerator.open(); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testUnknownField() { - String expectedExceptionMsg = "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + - " please check the field name."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(e.getMessage(), expectedExceptionMsg); - } - } - - @Test - public void testUnsupportedTimeType() { - String expectedExceptionMsg = "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + - "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedMultiParametersType() { - String expectedExceptionMsg = "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + - " only support single dynamic index pattern."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{local_date}-{local_time}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testDynamicIndexUnsupportedFormat() { - String expectedExceptionMsg = "Unsupported field: HourOfDay"; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); - } catch (UnsupportedTemporalTypeException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedIndexFieldType() { - String expectedExceptionMsg = "Unsupported type BOOLEAN of index field, Supported types are:" + - " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + - " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; - try { - IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); - } catch (IllegalArgumentException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + schema = + new TableSchema.Builder() + .field("id", DataTypes.INT()) + .field("item", DataTypes.STRING()) + .field("log_ts", DataTypes.BIGINT()) + .field("log_date", DataTypes.DATE()) + .field("log_time", DataTypes.TIME()) + .field("order_timestamp", DataTypes.TIMESTAMP()) + .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("status", DataTypes.BOOLEAN()) + .build(); + + rows = new ArrayList<>(); + rows.add( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-18").toEpochDay(), + (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), + TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), + true)); + rows.add( + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-19").toEpochDay(), + (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), + TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java index 96b6e3ec..bcfb68da 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java @@ -37,94 +37,99 @@ import static org.hamcrest.CoreMatchers.nullValue; import static org.junit.Assert.assertThat; -/** - * Tests for {@link KeyExtractor}. - */ +/** Tests for {@link KeyExtractor}. */ public class KeyExtractorTest { - @Test - public void testSimpleKey() { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .primaryKey("a") - .build(); - - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); - - String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); - assertThat(key, equalTo("12")); - } - - @Test - public void testNoPrimaryKey() { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .build(); - - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); - - String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); - assertThat(key, nullValue()); - } - - @Test - public void testTwoFieldsKey() { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .field("c", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "c") - .build(); - - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); - - String key = keyExtractor.apply( - GenericRowData.of( - 12L, - StringData.fromString("ABCD"), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12")) - )); - assertThat(key, equalTo("12_2012-12-12T12:12:12")); - } - - @Test - public void testAllTypesKey() { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.TINYINT().notNull()) - .field("b", DataTypes.SMALLINT().notNull()) - .field("c", DataTypes.INT().notNull()) - .field("d", DataTypes.BIGINT().notNull()) - .field("e", DataTypes.BOOLEAN().notNull()) - .field("f", DataTypes.FLOAT().notNull()) - .field("g", DataTypes.DOUBLE().notNull()) - .field("h", DataTypes.STRING().notNull()) - .field("i", DataTypes.TIMESTAMP().notNull()) - .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) - .field("k", DataTypes.TIME().notNull()) - .field("l", DataTypes.DATE().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") - .build(); - - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); - - String key = keyExtractor.apply( - GenericRowData.of( - (byte) 1, - (short) 2, - 3, - (long) 4, - true, - 1.0f, - 2.0d, - StringData.fromString("ABCD"), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12")), - TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), - (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), - (int) LocalDate.parse("2015-05-15").toEpochDay() - )); - assertThat( - key, - equalTo("1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); - } + @Test + public void testSimpleKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .primaryKey("a") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, equalTo("12")); + } + + @Test + public void testNoPrimaryKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, nullValue()); + } + + @Test + public void testTwoFieldsKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "c") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + assertThat(key, equalTo("12_2012-12-12T12:12:12")); + } + + @Test + public void testAllTypesKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.TINYINT().notNull()) + .field("b", DataTypes.SMALLINT().notNull()) + .field("c", DataTypes.INT().notNull()) + .field("d", DataTypes.BIGINT().notNull()) + .field("e", DataTypes.BOOLEAN().notNull()) + .field("f", DataTypes.FLOAT().notNull()) + .field("g", DataTypes.DOUBLE().notNull()) + .field("h", DataTypes.STRING().notNull()) + .field("i", DataTypes.TIMESTAMP().notNull()) + .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) + .field("k", DataTypes.TIME().notNull()) + .field("l", DataTypes.DATE().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + assertThat( + key, + equalTo( + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java index 32db4d35..5b72ce21 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -28,57 +28,51 @@ import java.util.HashMap; import java.util.Map; -/** - * A utility class for mocking {@link DynamicTableFactory.Context}. - */ +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ class TestContext { - private TableSchema schema; - private Map properties = new HashMap<>(); + private TableSchema schema; + private Map properties = new HashMap<>(); - public static TestContext context() { - return new TestContext(); - } + public static TestContext context() { + return new TestContext(); + } - public TestContext withSchema(TableSchema schema) { - this.schema = schema; - return this; - } + public TestContext withSchema(TableSchema schema) { + this.schema = schema; + return this; + } - DynamicTableFactory.Context build() { - return new DynamicTableFactory.Context() { - @Override - public ObjectIdentifier getObjectIdentifier() { - return null; - } + DynamicTableFactory.Context build() { + return new DynamicTableFactory.Context() { + @Override + public ObjectIdentifier getObjectIdentifier() { + return null; + } - @Override - public CatalogTable getCatalogTable() { - return new CatalogTableImpl( - schema, - properties, - "" - ); - } + @Override + public CatalogTable getCatalogTable() { + return new CatalogTableImpl(schema, properties, ""); + } - @Override - public ReadableConfig getConfiguration() { - return null; - } + @Override + public ReadableConfig getConfiguration() { + return null; + } - @Override - public ClassLoader getClassLoader() { - return TestContext.class.getClassLoader(); - } + @Override + public ClassLoader getClassLoader() { + return TestContext.class.getClassLoader(); + } - @Override - public boolean isTemporary() { - return false; - } - }; - } + @Override + public boolean isTemporary() { + return false; + } + }; + } - public TestContext withOption(String key, String value) { - properties.put(key, value); - return this; - } + public TestContext withOption(String key, String value) { + properties.put(key, value); + return this; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java index 68e0fe8a..cd11b70d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java @@ -36,73 +36,72 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.Assert.assertThat; -/** - * A resource that starts an embedded elasticsearch cluster. - */ +/** A resource that starts an embedded elasticsearch cluster. */ public class ElasticsearchResource extends ExternalResource { - private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchResource.class); - private EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; - private final TemporaryFolder tempFolder = new TemporaryFolder(); + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchResource.class); + private EmbeddedElasticsearchNodeEnvironment embeddedNodeEnv; + private final TemporaryFolder tempFolder = new TemporaryFolder(); - private final String clusterName; + private final String clusterName; - public ElasticsearchResource(String clusterName) { - this.clusterName = clusterName; - } + public ElasticsearchResource(String clusterName) { + this.clusterName = clusterName; + } - @Override - protected void before() throws Throwable { + @Override + protected void before() throws Throwable { - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Starting embedded Elasticsearch node "); - LOG.info("-------------------------------------------------------------------------"); + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Starting embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); - // dynamically load version-specific implementation of the Elasticsearch embedded node environment - Class clazz = Class.forName( - "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); - embeddedNodeEnv = (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); + // dynamically load version-specific implementation of the Elasticsearch embedded node + // environment + Class clazz = + Class.forName( + "org.apache.flink.streaming.connectors.elasticsearch.EmbeddedElasticsearchNodeEnvironmentImpl"); + embeddedNodeEnv = + (EmbeddedElasticsearchNodeEnvironment) InstantiationUtil.instantiate(clazz); - tempFolder.create(); - embeddedNodeEnv.start(tempFolder.newFolder(), clusterName); + tempFolder.create(); + embeddedNodeEnv.start(tempFolder.newFolder(), clusterName); - waitForCluster(); - } + waitForCluster(); + } - /** - * Blocks until the cluster is ready and data nodes/nodes are live. - */ - private void waitForCluster() { - AdminClient adminClient = embeddedNodeEnv.getClient().admin(); - ClusterAdminClient clusterAdminClient = adminClient.cluster(); + /** Blocks until the cluster is ready and data nodes/nodes are live. */ + private void waitForCluster() { + AdminClient adminClient = embeddedNodeEnv.getClient().admin(); + ClusterAdminClient clusterAdminClient = adminClient.cluster(); - ClusterHealthRequestBuilder requestBuilder = clusterAdminClient.prepareHealth("_all"); - requestBuilder = requestBuilder.setTimeout(TimeValue.timeValueSeconds(120)); + ClusterHealthRequestBuilder requestBuilder = clusterAdminClient.prepareHealth("_all"); + requestBuilder = requestBuilder.setTimeout(TimeValue.timeValueSeconds(120)); - ActionFuture healthFuture = - clusterAdminClient.health(requestBuilder.request()); + ActionFuture healthFuture = + clusterAdminClient.health(requestBuilder.request()); - ClusterHealthResponse health = healthFuture.actionGet(TimeValue.timeValueSeconds(120)); + ClusterHealthResponse health = healthFuture.actionGet(TimeValue.timeValueSeconds(120)); - assertThat(health.getNumberOfNodes(), greaterThanOrEqualTo(1)); - assertThat(health.getNumberOfDataNodes(), greaterThanOrEqualTo(1)); - } + assertThat(health.getNumberOfNodes(), greaterThanOrEqualTo(1)); + assertThat(health.getNumberOfDataNodes(), greaterThanOrEqualTo(1)); + } - @Override - protected void after() { + @Override + protected void after() { - LOG.info("-------------------------------------------------------------------------"); - LOG.info(" Shutting down embedded Elasticsearch node "); - LOG.info("-------------------------------------------------------------------------"); + LOG.info("-------------------------------------------------------------------------"); + LOG.info(" Shutting down embedded Elasticsearch node "); + LOG.info("-------------------------------------------------------------------------"); - try { - embeddedNodeEnv.close(); - tempFolder.delete(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } + try { + embeddedNodeEnv.close(); + tempFolder.delete(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } - public Client getClient() { - return embeddedNodeEnv.getClient(); - } + public Client getClient() { + return embeddedNodeEnv.getClient(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 7bcc7001..41e199de 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -37,105 +37,115 @@ import java.util.Map; /** - * This class contains utilities and a pre-defined source function and - * Elasticsearch Sink function used to simulate and verify data used in tests. + * This class contains utilities and a pre-defined source function and Elasticsearch Sink function + * used to simulate and verify data used in tests. */ public class SourceSinkDataTestKit { - private static final int NUM_ELEMENTS = 20; - - private static final String DATA_PREFIX = "message #"; - private static final String DATA_FIELD_NAME = "data"; - - private static final String TYPE_NAME = "flink-es-test-type"; - - /** - * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 - 20. - */ - public static class TestDataSourceFunction implements SourceFunction> { - private static final long serialVersionUID = 1L; - - private volatile boolean running = true; - - @Override - public void run(SourceFunction.SourceContext> ctx) throws Exception { - for (int i = 0; i < NUM_ELEMENTS && running; i++) { - ctx.collect(Tuple2.of(i, DATA_PREFIX + i)); - } - } - - @Override - public void cancel() { - running = false; - } - } - - public static ElasticsearchSinkFunction> getCborSinkFunction(String index) { - return new TestElasticsearchSinkFunction(index, XContentFactory::cborBuilder); - } - - public static ElasticsearchSinkFunction> getJsonSinkFunction(String index) { - return new TestElasticsearchSinkFunction(index, XContentFactory::jsonBuilder); - } - - public static ElasticsearchSinkFunction> getSmileSinkFunction(String index) { - return new TestElasticsearchSinkFunction(index, XContentFactory::smileBuilder); - } - - public static ElasticsearchSinkFunction> getYamlSinkFunction(String index) { - return new TestElasticsearchSinkFunction(index, XContentFactory::yamlBuilder); - } - - private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> { - private static final long serialVersionUID = 1L; - - private final String index; - private final XContentBuilderProvider contentBuilderProvider; - - /** - * Create the sink function, specifying a target Elasticsearch index. - * - * @param index Name of the target Elasticsearch index. - */ - public TestElasticsearchSinkFunction(String index, XContentBuilderProvider contentBuilderProvider) { - this.index = index; - this.contentBuilderProvider = contentBuilderProvider; - } - - public IndexRequest createIndexRequest(Tuple2 element) { - Map document = new HashMap<>(); - document.put(DATA_FIELD_NAME, element.f1); - - try { - return new IndexRequest(index, TYPE_NAME, element.f0.toString()).source(contentBuilderProvider.getBuilder().map(document)); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element)); - } - } - - /** - * Verify the results in an Elasticsearch index. The results must first be produced into the index - * using a {@link TestElasticsearchSinkFunction}; - * - * @param client The client to use to connect to Elasticsearch - * @param index The index to check - */ - public static void verifyProducedSinkData(Client client, String index) { - for (int i = 0; i < NUM_ELEMENTS; i++) { - GetResponse response = client.get(new GetRequest(index, TYPE_NAME, Integer.toString(i))).actionGet(); - Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); - } - } - - @FunctionalInterface - private interface XContentBuilderProvider extends Serializable { - XContentBuilder getBuilder() throws IOException; - } - + private static final int NUM_ELEMENTS = 20; + + private static final String DATA_PREFIX = "message #"; + private static final String DATA_FIELD_NAME = "data"; + + private static final String TYPE_NAME = "flink-es-test-type"; + + /** + * A {@link SourceFunction} that generates the elements (id, "message #" + id) with id being 0 - + * 20. + */ + public static class TestDataSourceFunction implements SourceFunction> { + private static final long serialVersionUID = 1L; + + private volatile boolean running = true; + + @Override + public void run(SourceFunction.SourceContext> ctx) + throws Exception { + for (int i = 0; i < NUM_ELEMENTS && running; i++) { + ctx.collect(Tuple2.of(i, DATA_PREFIX + i)); + } + } + + @Override + public void cancel() { + running = false; + } + } + + public static ElasticsearchSinkFunction> getCborSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::cborBuilder); + } + + public static ElasticsearchSinkFunction> getJsonSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::jsonBuilder); + } + + public static ElasticsearchSinkFunction> getSmileSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::smileBuilder); + } + + public static ElasticsearchSinkFunction> getYamlSinkFunction( + String index) { + return new TestElasticsearchSinkFunction(index, XContentFactory::yamlBuilder); + } + + private static class TestElasticsearchSinkFunction + implements ElasticsearchSinkFunction> { + private static final long serialVersionUID = 1L; + + private final String index; + private final XContentBuilderProvider contentBuilderProvider; + + /** + * Create the sink function, specifying a target Elasticsearch index. + * + * @param index Name of the target Elasticsearch index. + */ + public TestElasticsearchSinkFunction( + String index, XContentBuilderProvider contentBuilderProvider) { + this.index = index; + this.contentBuilderProvider = contentBuilderProvider; + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(DATA_FIELD_NAME, element.f1); + + try { + return new IndexRequest(index, TYPE_NAME, element.f0.toString()) + .source(contentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void process( + Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + } + + /** + * Verify the results in an Elasticsearch index. The results must first be produced into the + * index using a {@link TestElasticsearchSinkFunction}; + * + * @param client The client to use to connect to Elasticsearch + * @param index The index to check + */ + public static void verifyProducedSinkData(Client client, String index) { + for (int i = 0; i < NUM_ELEMENTS; i++) { + GetResponse response = + client.get(new GetRequest(index, TYPE_NAME, Integer.toString(i))).actionGet(); + Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java index b994f345..f96aece8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java @@ -30,131 +30,130 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; -/** - * Tests for the {@link Elasticsearch} descriptor. - */ +/** Tests for the {@link Elasticsearch} descriptor. */ public class ElasticsearchTest extends DescriptorTestBase { - @Test(expected = ValidationException.class) - public void testMissingIndex() { - removePropertyAndVerify(descriptors().get(0), "connector.index"); - } - - @Test(expected = ValidationException.class) - public void testInvalidFailureHandler() { - addPropertyAndVerify(descriptors().get(0), "connector.failure-handler", "invalid handler"); - } - - @Test(expected = ValidationException.class) - public void testInvalidMemorySize() { - addPropertyAndVerify(descriptors().get(1), "connector.bulk-flush.max-size", "12 bytes"); - } - - @Test(expected = ValidationException.class) - public void testInvalidProtocolInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "localhost:90"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Test(expected = ValidationException.class) - public void testInvalidHostNameInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "http://:90"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Test(expected = ValidationException.class) - public void testInvalidHostPortInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "http://localhost"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Override - public List descriptors() { - final Descriptor minimumDesc = new Elasticsearch() - .version("6") - .host("localhost", 1234, "http") - .index("MyIndex") - .documentType("MyType"); - - final Descriptor maximumDesc = - new Elasticsearch() - .version("6") - .host("host1", 1234, "https") - .host("host2", 1234, "https") - .index("MyIndex") - .documentType("MyType") - .keyDelimiter("#") - .keyNullLiteral("") - .bulkFlushBackoffExponential() - .bulkFlushBackoffDelay(123L) - .bulkFlushBackoffMaxRetries(3) - .bulkFlushInterval(100L) - .bulkFlushMaxActions(1000) - .bulkFlushMaxSize("12 MB") - .failureHandlerRetryRejected() - .connectionMaxRetryTimeout(100) - .connectionPathPrefix("/myapp"); - - final Descriptor customDesc = - new Elasticsearch() - .version("6") - .host("localhost", 1234, "http") - .index("MyIndex") - .documentType("MyType") - .disableFlushOnCheckpoint() - .failureHandlerCustom(NoOpFailureHandler.class); - - return Arrays.asList(minimumDesc, maximumDesc, customDesc); - } - - @Override - public List> properties() { - final Map minimumDesc = new HashMap<>(); - minimumDesc.put("connector.property-version", "1"); - minimumDesc.put("connector.type", "elasticsearch"); - minimumDesc.put("connector.version", "6"); - minimumDesc.put("connector.hosts", "http://localhost:1234"); - minimumDesc.put("connector.index", "MyIndex"); - minimumDesc.put("connector.document-type", "MyType"); - - final Map maximumDesc = new HashMap<>(); - maximumDesc.put("connector.property-version", "1"); - maximumDesc.put("connector.type", "elasticsearch"); - maximumDesc.put("connector.version", "6"); - maximumDesc.put("connector.hosts", "https://host1:1234;https://host2:1234"); - maximumDesc.put("connector.index", "MyIndex"); - maximumDesc.put("connector.document-type", "MyType"); - maximumDesc.put("connector.key-delimiter", "#"); - maximumDesc.put("connector.key-null-literal", ""); - maximumDesc.put("connector.bulk-flush.backoff.type", "exponential"); - maximumDesc.put("connector.bulk-flush.backoff.delay", "123"); - maximumDesc.put("connector.bulk-flush.backoff.max-retries", "3"); - maximumDesc.put("connector.bulk-flush.interval", "100"); - maximumDesc.put("connector.bulk-flush.max-actions", "1000"); - maximumDesc.put("connector.bulk-flush.max-size", "12 mb"); - maximumDesc.put("connector.failure-handler", "retry-rejected"); - maximumDesc.put("connector.connection-max-retry-timeout", "100"); - maximumDesc.put("connector.connection-path-prefix", "/myapp"); - - final Map customDesc = new HashMap<>(); - customDesc.put("connector.property-version", "1"); - customDesc.put("connector.type", "elasticsearch"); - customDesc.put("connector.version", "6"); - customDesc.put("connector.hosts", "http://localhost:1234"); - customDesc.put("connector.index", "MyIndex"); - customDesc.put("connector.document-type", "MyType"); - customDesc.put("connector.flush-on-checkpoint", "false"); - customDesc.put("connector.failure-handler", "custom"); - customDesc.put("connector.failure-handler-class", NoOpFailureHandler.class.getName()); - - return Arrays.asList(minimumDesc, maximumDesc, customDesc); - } - - @Override - public DescriptorValidator validator() { - return new ElasticsearchValidator(); - } + @Test(expected = ValidationException.class) + public void testMissingIndex() { + removePropertyAndVerify(descriptors().get(0), "connector.index"); + } + + @Test(expected = ValidationException.class) + public void testInvalidFailureHandler() { + addPropertyAndVerify(descriptors().get(0), "connector.failure-handler", "invalid handler"); + } + + @Test(expected = ValidationException.class) + public void testInvalidMemorySize() { + addPropertyAndVerify(descriptors().get(1), "connector.bulk-flush.max-size", "12 bytes"); + } + + @Test(expected = ValidationException.class) + public void testInvalidProtocolInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "localhost:90"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + + @Test(expected = ValidationException.class) + public void testInvalidHostNameInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "http://:90"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + + @Test(expected = ValidationException.class) + public void testInvalidHostPortInHosts() { + final DescriptorProperties descriptorProperties = new DescriptorProperties(); + descriptorProperties.putString(CONNECTOR_HOSTS, "http://localhost"); + ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); + } + + @Override + public List descriptors() { + final Descriptor minimumDesc = + new Elasticsearch() + .version("6") + .host("localhost", 1234, "http") + .index("MyIndex") + .documentType("MyType"); + + final Descriptor maximumDesc = + new Elasticsearch() + .version("6") + .host("host1", 1234, "https") + .host("host2", 1234, "https") + .index("MyIndex") + .documentType("MyType") + .keyDelimiter("#") + .keyNullLiteral("") + .bulkFlushBackoffExponential() + .bulkFlushBackoffDelay(123L) + .bulkFlushBackoffMaxRetries(3) + .bulkFlushInterval(100L) + .bulkFlushMaxActions(1000) + .bulkFlushMaxSize("12 MB") + .failureHandlerRetryRejected() + .connectionMaxRetryTimeout(100) + .connectionPathPrefix("/myapp"); + + final Descriptor customDesc = + new Elasticsearch() + .version("6") + .host("localhost", 1234, "http") + .index("MyIndex") + .documentType("MyType") + .disableFlushOnCheckpoint() + .failureHandlerCustom(NoOpFailureHandler.class); + + return Arrays.asList(minimumDesc, maximumDesc, customDesc); + } + + @Override + public List> properties() { + final Map minimumDesc = new HashMap<>(); + minimumDesc.put("connector.property-version", "1"); + minimumDesc.put("connector.type", "elasticsearch"); + minimumDesc.put("connector.version", "6"); + minimumDesc.put("connector.hosts", "http://localhost:1234"); + minimumDesc.put("connector.index", "MyIndex"); + minimumDesc.put("connector.document-type", "MyType"); + + final Map maximumDesc = new HashMap<>(); + maximumDesc.put("connector.property-version", "1"); + maximumDesc.put("connector.type", "elasticsearch"); + maximumDesc.put("connector.version", "6"); + maximumDesc.put("connector.hosts", "https://host1:1234;https://host2:1234"); + maximumDesc.put("connector.index", "MyIndex"); + maximumDesc.put("connector.document-type", "MyType"); + maximumDesc.put("connector.key-delimiter", "#"); + maximumDesc.put("connector.key-null-literal", ""); + maximumDesc.put("connector.bulk-flush.backoff.type", "exponential"); + maximumDesc.put("connector.bulk-flush.backoff.delay", "123"); + maximumDesc.put("connector.bulk-flush.backoff.max-retries", "3"); + maximumDesc.put("connector.bulk-flush.interval", "100"); + maximumDesc.put("connector.bulk-flush.max-actions", "1000"); + maximumDesc.put("connector.bulk-flush.max-size", "12 mb"); + maximumDesc.put("connector.failure-handler", "retry-rejected"); + maximumDesc.put("connector.connection-max-retry-timeout", "100"); + maximumDesc.put("connector.connection-path-prefix", "/myapp"); + + final Map customDesc = new HashMap<>(); + customDesc.put("connector.property-version", "1"); + customDesc.put("connector.type", "elasticsearch"); + customDesc.put("connector.version", "6"); + customDesc.put("connector.hosts", "http://localhost:1234"); + customDesc.put("connector.index", "MyIndex"); + customDesc.put("connector.document-type", "MyType"); + customDesc.put("connector.flush-on-checkpoint", "false"); + customDesc.put("connector.failure-handler", "custom"); + customDesc.put("connector.failure-handler-class", NoOpFailureHandler.class.getName()); + + return Arrays.asList(minimumDesc, maximumDesc, customDesc); + } + + @Override + public DescriptorValidator validator() { + return new ElasticsearchValidator(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java index de14831e..af03eedc 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -29,52 +29,51 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; -/** - * Elasticsearch 6 specific configuration. - */ +/** Elasticsearch 6 specific configuration. */ @Internal final class Elasticsearch6Configuration extends ElasticsearchConfiguration { - Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { - super(config, classLoader); - } + Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } - public List getHosts() { - return config.get(HOSTS_OPTION).stream() - .map(Elasticsearch6Configuration::validateAndParseHostsString) - .collect(Collectors.toList()); - } + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch6Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } - /** - * Parse Hosts String to list. - * - *

Hosts String format was given as following: - * - *

-	 *     connector.hosts = http://host_name:9092;http://host_name:9093
-	 * 
- */ - private static HttpHost validateAndParseHostsString(String host) { - try { - HttpHost httpHost = HttpHost.create(host); - if (httpHost.getPort() < 0) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", - host, - HOSTS_OPTION.key())); - } + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+     *     connector.hosts = http://host_name:9092;http://host_name:9093
+     * 
+ */ + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } - if (httpHost.getSchemeName() == null) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", - host, - HOSTS_OPTION.key())); - } - return httpHost; - } catch (Exception e) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", - host, - HOSTS_OPTION.key()), e); - } - } + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index d4e5866c..508a626e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -49,266 +49,269 @@ import java.util.Objects; /** - * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a logical - * description. + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. */ @PublicEvolving final class Elasticsearch6DynamicSink implements DynamicTableSink { - @VisibleForTesting - static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); - - private final EncodingFormat> format; - private final TableSchema schema; - private final Elasticsearch6Configuration config; - - public Elasticsearch6DynamicSink( - EncodingFormat> format, - Elasticsearch6Configuration config, - TableSchema schema) { - this(format, config, schema, (ElasticsearchSink.Builder::new)); - } - - //-------------------------------------------------------------- - // Hack to make configuration testing possible. - // - // The code in this block should never be used outside of tests. - // Having a way to inject a builder we can assert the builder in - // the test. We can not assert everything though, e.g. it is not - // possible to assert flushing on checkpoint, as it is configured - // on the sink itself. - //-------------------------------------------------------------- - - private final ElasticSearchBuilderProvider builderProvider; - - @FunctionalInterface - interface ElasticSearchBuilderProvider { - ElasticsearchSink.Builder createBuilder( - List httpHosts, - RowElasticsearchSinkFunction upsertSinkFunction); - } - - Elasticsearch6DynamicSink( - EncodingFormat> format, - Elasticsearch6Configuration config, - TableSchema schema, - ElasticSearchBuilderProvider builderProvider) { - this.format = format; - this.schema = schema; - this.config = config; - this.builderProvider = builderProvider; - } - - //-------------------------------------------------------------- - // End of hack to make configuration testing possible - //-------------------------------------------------------------- - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - if (kind != RowKind.UPDATE_BEFORE) { - builder.addContainedKind(kind); - } - } - return builder.build(); - } - - @Override - public SinkFunctionProvider getSinkRuntimeProvider(Context context) { - return () -> { - SerializationSchema format = this.format.createRuntimeEncoder(context, schema.toRowDataType()); - - final RowElasticsearchSinkFunction upsertFunction = - new RowElasticsearchSinkFunction( - IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), - config.getDocumentType(), - format, - XContentType.JSON, - REQUEST_FACTORY, - KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()) - ); - - final ElasticsearchSink.Builder builder = builderProvider.createBuilder( - config.getHosts(), - upsertFunction); - - builder.setFailureHandler(config.getFailureHandler()); - builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); - builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); - builder.setBulkFlushInterval(config.getBulkFlushInterval()); - builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); - config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); - config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); - config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); - - // we must overwrite the default factory which is defined with a lambda because of a bug - // in shading lambda serialization shading see FLINK-18006 - if (config.getUsername().isPresent() - && config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { - builder.setRestClientFactory(new AuthRestClientFactory(config.getPathPrefix().orElse(null), config.getUsername().get(), config.getPassword().get())); - } else { - builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); - } - - final ElasticsearchSink sink = builder.build(); - - if (config.isDisableFlushOnCheckpoint()) { - sink.disableFlushOnCheckpoint(); - } - - return sink; - }; - } - - @Override - public DynamicTableSink copy() { - return this; - } - - @Override - public String asSummaryString() { - return "Elasticsearch6"; - } - - /** - * Serializable {@link RestClientFactory} used by the sink. - */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** - * Serializable {@link RestClientFactory} used by the sink which enable authentication. - */ - @VisibleForTesting - static class AuthRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - private final String username; - private final String password; - private transient CredentialsProvider credentialsProvider; - - public AuthRestClientFactory(@Nullable String pathPrefix, String username, String password) { - this.pathPrefix = pathPrefix; - this.password = password; - this.username = username; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - if (credentialsProvider == null) { - credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); - } - restClientBuilder.setHttpClientConfigCallback(httpAsyncClientBuilder -> - httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AuthRestClientFactory that = (AuthRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix) && - Objects.equals(username, that.username) && - Objects.equals(password, that.password); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix, username, password); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. - */ - private static class Elasticsearch6RequestFactory implements RequestFactory { - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, docType, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new IndexRequest(index, docType, key) - .source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, docType, key); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; - return Objects.equals(format, that.format) && - Objects.equals(schema, that.schema) && - Objects.equals(config, that.config) && - Objects.equals(builderProvider, that.builderProvider); - } - - @Override - public int hashCode() { - return Objects.hash(format, schema, config, builderProvider); - } + @VisibleForTesting + static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch6Configuration config; + + public Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema) { + this(format, config, schema, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), + config.getDocumentType(), + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch6"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, username, password); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch6RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index, docType, key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 5fb57697..6fc4030c 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -56,119 +56,115 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; -/** - * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. - */ +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ @Internal public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { - private static final Set> requiredOptions = Stream.of( - HOSTS_OPTION, - INDEX_OPTION, - DOCUMENT_TYPE_OPTION - ).collect(Collectors.toSet()); - private static final Set> optionalOptions = Stream.of( - KEY_DELIMITER_OPTION, - FAILURE_HANDLER_OPTION, - FLUSH_ON_CHECKPOINT_OPTION, - BULK_FLASH_MAX_SIZE_OPTION, - BULK_FLUSH_MAX_ACTIONS_OPTION, - BULK_FLUSH_INTERVAL_OPTION, - BULK_FLUSH_BACKOFF_TYPE_OPTION, - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, - BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, - CONNECTION_PATH_PREFIX, - FORMAT_OPTION, - PASSWORD_OPTION, - USERNAME_OPTION - ).collect(Collectors.toSet()); + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION, DOCUMENT_TYPE_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - TableSchema tableSchema = context.getCatalogTable().getSchema(); - ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); - final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); - final EncodingFormat> format = helper.discoverEncodingFormat( - SerializationFormatFactory.class, - FORMAT_OPTION); + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); - helper.validate(); - Configuration configuration = new Configuration(); - context.getCatalogTable() - .getOptions() - .forEach(configuration::setString); - Elasticsearch6Configuration config = new Elasticsearch6Configuration(configuration, context.getClassLoader()); + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch6Configuration config = + new Elasticsearch6Configuration(configuration, context.getClassLoader()); - validate(config, configuration); + validate(config, configuration); - return new Elasticsearch6DynamicSink( - format, - config, - TableSchemaUtils.getPhysicalSchema(tableSchema)); - } + return new Elasticsearch6DynamicSink( + format, config, TableSchemaUtils.getPhysicalSchema(tableSchema)); + } - private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { - config.getFailureHandler(); // checks if we can instantiate the custom failure handler - config.getHosts(); // validate hosts - validate( - config.getIndex().length() >= 1, - () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); - int maxActions = config.getBulkFlushMaxActions(); - validate( - maxActions == -1 || maxActions >= 1, - () -> String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - maxActions) - ); - long maxSize = config.getBulkFlushMaxByteSize(); - long mb1 = 1024 * 1024; - validate( - maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), - () -> String.format( - "'%s' must be in MB granularity. Got: %s", - BULK_FLASH_MAX_SIZE_OPTION.key(), - originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) - ); - validate( - config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), - () -> String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), - config.getBulkFlushBackoffRetries().get()) - ); - if (config.getUsername().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { - validate( - config.getPassword().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), - () -> String.format( - "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", - USERNAME_OPTION.key(), - PASSWORD_OPTION.key(), - config.getUsername().get(), - config.getPassword().orElse("") - )); - } - } + private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } - private static void validate(boolean condition, Supplier message) { - if (!condition) { - throw new ValidationException(message.get()); - } - } + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } - @Override - public String factoryIdentifier() { - return "elasticsearch-6"; - } + @Override + public String factoryIdentifier() { + return "elasticsearch-6"; + } - @Override - public Set> requiredOptions() { - return requiredOptions; - } + @Override + public Set> requiredOptions() { + return requiredOptions; + } - @Override - public Set> optionalOptions() { - return optionalOptions; - } + @Override + public Set> optionalOptions() { + return optionalOptions; + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 24f97f68..89d1e7d8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -41,105 +41,102 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; -/** - * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. - */ +/** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ @Internal -public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge { - - private static final long serialVersionUID = -5222683870097809633L; - - private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6ApiCallBridge.class); - - /** - * User-provided HTTP Host. - */ - private final List httpHosts; - - /** - * The factory to configure the rest client. - */ - private final RestClientFactory restClientFactory; - - Elasticsearch6ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { - Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); - this.httpHosts = httpHosts; - this.restClientFactory = Preconditions.checkNotNull(restClientFactory); - } - - @Override - public RestHighLevelClient createClient(Map clientConfig) { - RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); - restClientFactory.configureRestClientBuilder(builder); - - RestHighLevelClient rhlClient = new RestHighLevelClient(builder); - - return rhlClient; - } - - @Override - public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { - return BulkProcessor.builder(client::bulkAsync, listener); - } - - @Override - public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { - if (!bulkItemResponse.isFailed()) { - return null; - } else { - return bulkItemResponse.getFailure().getCause(); - } - } - - @Override - public void configureBulkProcessorBackoff( - BulkProcessor.Builder builder, - @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { - - BackoffPolicy backoffPolicy; - if (flushBackoffPolicy != null) { - switch (flushBackoffPolicy.getBackoffType()) { - case CONSTANT: - backoffPolicy = BackoffPolicy.constantBackoff( - new TimeValue(flushBackoffPolicy.getDelayMillis()), - flushBackoffPolicy.getMaxRetryCount()); - break; - case EXPONENTIAL: - default: - backoffPolicy = BackoffPolicy.exponentialBackoff( - new TimeValue(flushBackoffPolicy.getDelayMillis()), - flushBackoffPolicy.getMaxRetryCount()); - } - } else { - backoffPolicy = BackoffPolicy.noBackoff(); - } - - builder.setBackoffPolicy(backoffPolicy); - } - - @Override - public RequestIndexer createBulkProcessorIndexer( - BulkProcessor bulkProcessor, - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new Elasticsearch6BulkProcessorIndexer( - bulkProcessor, - flushOnCheckpoint, - numPendingRequestsRef); - } - - @Override - public void verifyClientConnection(RestHighLevelClient client) throws IOException { - if (LOG.isInfoEnabled()) { - LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); - } - - if (!client.ping()) { - throw new RuntimeException("There are no reachable Elasticsearch nodes!"); - } - - if (LOG.isInfoEnabled()) { - LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); - } - } +public class Elasticsearch6ApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6ApiCallBridge.class); + + /** User-provided HTTP Host. */ + private final List httpHosts; + + /** The factory to configure the rest client. */ + private final RestClientFactory restClientFactory; + + Elasticsearch6ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) { + RestClientBuilder builder = + RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client::bulkAsync, listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = + BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = + BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch6BulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping()) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java index af3c5b13..dfeb54ad 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java @@ -32,54 +32,54 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. - * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. * *

Note: This class is binary compatible to Elasticsearch 6. */ @Internal class Elasticsearch6BulkProcessorIndexer implements RequestIndexer { - private final BulkProcessor bulkProcessor; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; - Elasticsearch6BulkProcessorIndexer( - BulkProcessor bulkProcessor, - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - this.bulkProcessor = checkNotNull(bulkProcessor); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } + Elasticsearch6BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(deleteRequest); - } - } + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(indexRequest); - } - } + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(updateRequest); - } - } + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java index 4149d14c..1a6a547b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java @@ -56,214 +56,210 @@ import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_MAX_RETRY_TIMEOUT; import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; -/** - * Version-specific upsert table sink for Elasticsearch 6. - */ +/** Version-specific upsert table sink for Elasticsearch 6. */ @Internal public class Elasticsearch6UpsertTableSink extends ElasticsearchUpsertTableSinkBase { - @VisibleForTesting - static final RequestFactory UPDATE_REQUEST_FACTORY = - new Elasticsearch6RequestFactory(); - - public Elasticsearch6UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @Override - protected ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertSinkFunction) { - - final List httpHosts = hosts.stream() - .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) - .collect(Collectors.toList()); - - final ElasticsearchSink.Builder> builder = createBuilder(upsertSinkFunction, httpHosts); - - builder.setFailureHandler(failureHandler); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) - .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) - .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) - .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) - .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) - .ifPresent(v -> builder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) - .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) - .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); - - builder.setRestClientFactory( - new DefaultRestClientFactory( - Optional.ofNullable(sinkOptions.get(REST_MAX_RETRY_TIMEOUT)) - .map(Integer::valueOf) - .orElse(null), - sinkOptions.get(REST_PATH_PREFIX))); - - final ElasticsearchSink> sink = builder.build(); - - Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) - .ifPresent(v -> { - if (Boolean.valueOf(v)) { - sink.disableFlushOnCheckpoint(); - } - }); - - return sink; - } - - @VisibleForTesting - ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, - List httpHosts) { - return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** - * Serializable {@link RestClientFactory} used by the sink. - */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private Integer maxRetryTimeout; - private String pathPrefix; - - public DefaultRestClientFactory(@Nullable Integer maxRetryTimeout, @Nullable String pathPrefix) { - this.maxRetryTimeout = maxRetryTimeout; - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (maxRetryTimeout != null) { - restClientBuilder.setMaxRetryTimeoutMillis(maxRetryTimeout); - } - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(maxRetryTimeout, that.maxRetryTimeout) && - Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash( - maxRetryTimeout, - pathPrefix); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. - */ - private static class Elasticsearch6RequestFactory implements RequestFactory { - - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, docType, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - XContentType contentType, - byte[] document) { - return new IndexRequest(index, docType) - .source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, docType, key); - } - } + @VisibleForTesting + static final RequestFactory UPDATE_REQUEST_FACTORY = new Elasticsearch6RequestFactory(); + + public Elasticsearch6UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @Override + protected ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + return new Elasticsearch6UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertSinkFunction) { + + final List httpHosts = + hosts.stream() + .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) + .collect(Collectors.toList()); + + final ElasticsearchSink.Builder> builder = + createBuilder(upsertSinkFunction, httpHosts); + + builder.setFailureHandler(failureHandler); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) + .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) + .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) + .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) + .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) + .ifPresent( + v -> + builder.setBulkFlushBackoffType( + ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) + .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) + .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); + + builder.setRestClientFactory( + new DefaultRestClientFactory( + Optional.ofNullable(sinkOptions.get(REST_MAX_RETRY_TIMEOUT)) + .map(Integer::valueOf) + .orElse(null), + sinkOptions.get(REST_PATH_PREFIX))); + + final ElasticsearchSink> sink = builder.build(); + + Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) + .ifPresent( + v -> { + if (Boolean.valueOf(v)) { + sink.disableFlushOnCheckpoint(); + } + }); + + return sink; + } + + @VisibleForTesting + ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { + return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private Integer maxRetryTimeout; + private String pathPrefix; + + public DefaultRestClientFactory( + @Nullable Integer maxRetryTimeout, @Nullable String pathPrefix) { + this.maxRetryTimeout = maxRetryTimeout; + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (maxRetryTimeout != null) { + restClientBuilder.setMaxRetryTimeoutMillis(maxRetryTimeout); + } + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(maxRetryTimeout, that.maxRetryTimeout) + && Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(maxRetryTimeout, pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch6RequestFactory implements RequestFactory { + + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, String docType, XContentType contentType, byte[] document) { + return new IndexRequest(index, docType).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java index 1b9b142d..048a9f13 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java @@ -36,42 +36,40 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_6; -/** - * Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 6. - */ +/** Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 6. */ @Internal public class Elasticsearch6UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { - @Override - protected String elasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_6; - } + @Override + protected String elasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_6; + } - @Override - protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { + @Override + protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } + return new Elasticsearch6UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index c9465a22..b899be06 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -35,192 +35,215 @@ import java.util.Objects; /** - * Elasticsearch 6.x sink that requests multiple {@link ActionRequest ActionRequests} - * against a cluster for each incoming element. + * Elasticsearch 6.x sink that requests multiple {@link ActionRequest ActionRequests} against a + * cluster for each incoming element. * - *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. - * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch + * cluster. The sink will fail if no cluster can be connected to using the provided transport + * addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest + * ActionRequests}. This will buffer elements before sending a request to the cluster. The behaviour + * of the {@code BulkProcessor} can be configured using these config keys: * - *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. - * This will buffer elements before sending a request to the cluster. The behaviour of the - * {@code BulkProcessor} can be configured using these config keys: *

    - *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer - *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer - *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two - * settings in milliseconds + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds *
* *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple - * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of - * {@link ElasticsearchSinkFunction} for an example. + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation + * of {@link ElasticsearchSinkFunction} for an example. * * @param Type of the elements handled by this sink */ @PublicEvolving public class ElasticsearchSink extends ElasticsearchSinkBase { - private static final long serialVersionUID = 1L; - - private ElasticsearchSink( - Map bulkRequestsConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler, - RestClientFactory restClientFactory) { - - super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); - } - - /** - * A builder for creating an {@link ElasticsearchSink}. - * - * @param Type of the elements handled by the sink this builder creates. - */ - @PublicEvolving - public static class Builder { - - private final List httpHosts; - private final ElasticsearchSinkFunction elasticsearchSinkFunction; - - private Map bulkRequestsConfig = new HashMap<>(); - private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); - private RestClientFactory restClientFactory = restClientBuilder -> {}; - - /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. - * - * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. - */ - public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { - this.httpHosts = Preconditions.checkNotNull(httpHosts); - this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); - } - - /** - * Sets the maximum number of actions to buffer for each bulk request. - * - * @param numMaxActions the maximum number of actions to buffer per bulk request. - */ - public void setBulkFlushMaxActions(int numMaxActions) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); - } - - /** - * Sets the maximum size of buffered actions, in mb, per bulk request. - * - * @param maxSizeMb the maximum size of buffered actions, in mb. - */ - public void setBulkFlushMaxSizeMb(int maxSizeMb) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); - } - - /** - * Sets the bulk flush interval, in milliseconds. - * - * @param intervalMillis the bulk flush interval, in milliseconds. - */ - public void setBulkFlushInterval(long intervalMillis) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); - } - - /** - * Sets whether or not to enable bulk flush backoff behaviour. - * - * @param enabled whether or not to enable backoffs. - */ - public void setBulkFlushBackoff(boolean enabled) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); - } - - /** - * Sets the type of back of to use when flushing bulk requests. - * - * @param flushBackoffType the backoff type to use. - */ - public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { - this.bulkRequestsConfig.put( - CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, - Preconditions.checkNotNull(flushBackoffType).toString()); - } - - /** - * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. - * - * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests - */ - public void setBulkFlushBackoffRetries(int maxRetries) { - Preconditions.checkArgument( - maxRetries > 0, - "Max number of backoff attempts must be larger than 0."); - - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); - } - - /** - * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. - * - * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. - */ - public void setBulkFlushBackoffDelay(long delayMillis) { - Preconditions.checkArgument( - delayMillis >= 0, - "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); - } - - /** - * Sets a failure handler for action requests. - * - * @param failureHandler This is used to handle failed {@link ActionRequest}. - */ - public void setFailureHandler(ActionRequestFailureHandler failureHandler) { - this.failureHandler = Preconditions.checkNotNull(failureHandler); - } - - /** - * Sets a REST client factory for custom client configuration. - * - * @param restClientFactory the factory that configures the rest client. - */ - public void setRestClientFactory(RestClientFactory restClientFactory) { - this.restClientFactory = Preconditions.checkNotNull(restClientFactory); - } - - /** - * Creates the Elasticsearch sink. - * - * @return the created Elasticsearch sink. - */ - public ElasticsearchSink build() { - return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Builder builder = (Builder) o; - return Objects.equals(httpHosts, builder.httpHosts) && - Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) && - Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) && - Objects.equals(failureHandler, builder.failureHandler) && - Objects.equals(restClientFactory, builder.restClientFactory); - } - - @Override - public int hashCode() { - return Objects.hash( - httpHosts, - elasticsearchSinkFunction, - bulkRequestsConfig, - failureHandler, - restClientFactory); - } - } + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super( + new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), + bulkRequestsConfig, + elasticsearchSinkFunction, + failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + */ + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link + * RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} + * from the incoming element. + */ + public Builder( + List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>( + bulkRequestsConfig, + httpHosts, + elasticsearchSinkFunction, + failureHandler, + restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) + && Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) + && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) + && Objects.equals(failureHandler, builder.failureHandler) + && Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java index 4b74649c..67246bb6 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java @@ -24,17 +24,16 @@ import java.io.Serializable; /** - * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally - * used in the {@link ElasticsearchSink}. + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} + * internally used in the {@link ElasticsearchSink}. */ @PublicEvolving public interface RestClientFactory extends Serializable { - /** - * Configures the rest client builder. - * - * @param restClientBuilder the configured rest client builder. - */ - void configureRestClientBuilder(RestClientBuilder restClientBuilder); - + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 3d90c890..0d1f7510 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -29,200 +29,195 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; -/** - * Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. - */ +/** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ public class Elasticsearch6DynamicSinkFactoryTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Test - public void validateEmptyConfiguration() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "One or more required options are missing.\n" + - "\n" + - "Missing required options are:\n" + - "\n" + - "document-type\n" + - "hosts\n" + - "index"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .build() - ); - } - - @Test - public void validateWrongIndex() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'index' must not be empty"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption("index", "") - .withOption("document-type", "MyType") - .withOption("hosts", "http://localhost:12345") - .build() - ); - } - - @Test - public void validateWrongHosts() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption("index", "MyIndex") - .withOption("document-type", "MyType") - .withOption("hosts", "wrong-host") - .build() - ); - } - - @Test - public void validateWrongFlushSize() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") - .build() - ); - } - - @Test - public void validateWrongRetries() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "0") - .build() - ); - } - - @Test - public void validateWrongMaxActions() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") - .build() - ); - } - - @Test - public void validateWrongBackoffDelay() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Invalid value for option 'sink.bulk-flush.backoff.delay'."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") - .build() - ); - } - - @Test - public void validatePrimaryKeyOnIllegalColumn() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "The table has a primary key on columns of illegal types: " + - "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + - " Elasticsearch sink does not support primary keys on columns of types: " + - "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.ARRAY(DataTypes.BIGINT().notNull()).notNull()) - .field("c", DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING()).notNull()) - .field("d", DataTypes.MULTISET(DataTypes.BIGINT().notNull()).notNull()) - .field("e", DataTypes.ROW(DataTypes.FIELD("a", DataTypes.BIGINT())).notNull()) - .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) - .field("g", DataTypes.BYTES().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g") - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") - .build() - ); - } - - @Test - public void validateWrongCredential() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") - .build() - ); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption("index", "") + .withOption("document-type", "MyType") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption("index", "MyIndex") + .withOption("document-type", "MyType") + .withOption("hosts", "wrong-host") + .build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field( + "b", + DataTypes.ARRAY(DataTypes.BIGINT().notNull()) + .notNull()) + .field( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()) + .field( + "d", + DataTypes.MULTISET(DataTypes.BIGINT().notNull()) + .notNull()) + .field( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", DataTypes.BIGINT())) + .notNull()) + .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field("g", DataTypes.BYTES().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g") + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + .build()); + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index f1604d0d..cb8457b9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -59,272 +59,331 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -/** - * IT tests for {@link Elasticsearch6DynamicSink}. - */ +/** IT tests for {@link Elasticsearch6DynamicSink}. */ public class Elasticsearch6DynamicSinkITCase { - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName - .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("6.3.1")); + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("6.3.1")); - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings) - .addTransportAddress(transportAddress); - } + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } - @Test - public void testWritingDocuments() throws Exception { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.TIME()) - .field("c", DataTypes.STRING().notNull()) - .field("d", DataTypes.FLOAT()) - .field("e", DataTypes.TINYINT().notNull()) - .field("f", DataTypes.DATE()) - .field("g", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "g") - .build(); - GenericRowData rowData = GenericRowData.of( - 1L, - 12345, - StringData.fromString("ABCDE"), - 12.12f, - (byte) 2, - 12345, - TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12"))); + @Test + public void testWritingDocuments() throws Exception { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.TIME()) + .field("c", DataTypes.STRING().notNull()) + .field("d", DataTypes.FLOAT()) + .field("e", DataTypes.TINYINT().notNull()) + .field("f", DataTypes.DATE()) + .field("g", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "g") + .build(); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); - String index = "writing-documents"; - String myType = "MyType"; - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + String index = "writing-documents"; + String myType = "MyType"; + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - SinkFunctionProvider sinkRuntimeProvider = (SinkFunctionProvider) sinkFactory.createDynamicTableSink( - context() - .withSchema(schema) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) - .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") - .build() - ).getSinkRuntimeProvider(new MockContext()); + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchOptions.INDEX_OPTION.key(), + index) + .withOption( + ElasticsearchOptions.DOCUMENT_TYPE_OPTION + .key(), + myType) + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); - SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); - StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); - rowData.setRowKind(RowKind.UPDATE_AFTER); - environment.fromElements(rowData).addSink(sinkFunction); - environment.execute(); + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); - Client client = getClient(); - Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - @Test - public void testWritingDocumentsFromTableApi() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "table-api"; - String myType = "MyType"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIME,\n" + - "c STRING NOT NULL,\n" + - "d FLOAT,\n" + - "e TINYINT NOT NULL,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL,\n" + - "h as a + 2,\n" + - "PRIMARY KEY (a, g) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "table-api"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")) - ).executeInsert("esTable").await(); + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); - Map response = client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) - .actionGet() - .getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - @Test - public void testWritingDocumentsNoPrimaryKey() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "no-primary-key"; - String myType = "MyType"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIME,\n" + - "c STRING NOT NULL,\n" + - "d FLOAT,\n" + - "e TINYINT NOT NULL,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "no-primary-key"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 2L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "FGHIJK", - 13.13f, - (byte) 4, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2013-12-12T13:13:13")) - ).executeInsert("esTable").await(); + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); + Client client = getClient(); - // search API does not return documents that were not indexed, we might need to query - // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); - SearchHits hits; - do { - hits = client.prepareSearch(index) - .execute() - .actionGet() - .getHits(); - if (hits.getTotalHits() < 2) { - Thread.sleep(200); - } - } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits() < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); - if (hits.getTotalHits() < 2) { - throw new AssertionError("Could not retrieve results from Elasticsearch."); - } + if (hits.getTotalHits() < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } - HashSet> resultSet = new HashSet<>(); - resultSet.add(hits.getAt(0).getSourceAsMap()); - resultSet.add(hits.getAt(1).getSourceAsMap()); - Map expectedMap1 = new HashMap<>(); - expectedMap1.put("a", 1); - expectedMap1.put("b", "00:00:12"); - expectedMap1.put("c", "ABCDE"); - expectedMap1.put("d", 12.12d); - expectedMap1.put("e", 2); - expectedMap1.put("f", "2003-10-20"); - expectedMap1.put("g", "2012-12-12 12:12:12"); - Map expectedMap2 = new HashMap<>(); - expectedMap2.put("a", 2); - expectedMap2.put("b", "00:00:12"); - expectedMap2.put("c", "FGHIJK"); - expectedMap2.put("d", 13.13d); - expectedMap2.put("e", 4); - expectedMap2.put("f", "2003-10-20"); - expectedMap2.put("g", "2013-12-12 13:13:13"); - HashSet> expectedSet = new HashSet<>(); - expectedSet.add(expectedMap1); - expectedSet.add(expectedMap2); - assertThat(resultSet, equalTo(expectedSet)); - } + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } - @Test - public void testWritingDocumentsWithDynamicIndex() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "dynamic-index-{b|yyyy-MM-dd}"; - String myType = "MyType"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIMESTAMP NOT NULL,\n" + - "PRIMARY KEY (a) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "dynamic-index-{b|yyyy-MM-dd}"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable").await(); + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); - Map response = client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - private static class MockContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { - return null; - } - } + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index 0e54ed54..5d585933 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -45,211 +45,234 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; -/** - * Tests for {@link Elasticsearch6DynamicSink} parameters. - */ +/** Tests for {@link Elasticsearch6DynamicSink} parameters. */ public class Elasticsearch6DynamicSinkTest { - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - private static final String HOSTNAME = "host1"; - private static final int PORT = 1234; - private static final String SCHEMA = "https"; - private static final String INDEX = "MyIndex"; - private static final String DOC_TYPE = "MyType"; - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration(getConfig(), this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(true); - verify(provider.builderSpy).setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - verify(provider.builderSpy).setBulkFlushBackoffDelay(123); - verify(provider.builderSpy).setBulkFlushBackoffRetries(3); - verify(provider.builderSpy).setBulkFlushInterval(100); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); - verify(provider.sinkSpy).disableFlushOnCheckpoint(); - } - - @Test - public void testDefaultConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration(configuration, this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - @Test - public void testAuthConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration(configuration, this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch6DynamicSink.AuthRestClientFactory(null, USERNAME, PASSWORD)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - private Configuration getConfig() { - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); - configuration.setString(ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); - configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); - return configuration; - } - - private static class BuilderProvider implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { - public ElasticsearchSink.Builder builderSpy; - public ElasticsearchSink sinkSpy; - - @Override - public ElasticsearchSink.Builder createBuilder( - List httpHosts, - RowElasticsearchSinkFunction upsertSinkFunction) { - builderSpy = Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); - doAnswer( - invocation -> { - sinkSpy = Mockito.spy((ElasticsearchSink) invocation.callRealMethod()); - return sinkSpy; - } - ).when(builderSpy).build(); - - return builderSpy; - } - } - - private TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - private static class DummySerializationSchema implements SerializationSchema { - - private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); - - @Override - public byte[] serialize(RowData element) { - return new byte[0]; - } - } - - private static class DummyEncodingFormat implements EncodingFormat> { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, - DataType consumedDataType) { - return DummySerializationSchema.INSTANCE; - } - - @Override - public ChangelogMode getChangelogMode() { - return null; - } - } - - private static class MockSinkContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { - return null; - } - } - - /** - * Custom failure handler for testing. - */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java index 29e9d491..743040d8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java @@ -55,171 +55,184 @@ import static org.junit.Assert.assertEquals; /** - * Test for {@link Elasticsearch6UpsertTableSink} created by {@link Elasticsearch6UpsertTableSinkFactory}. + * Test for {@link Elasticsearch6UpsertTableSink} created by {@link + * Elasticsearch6UpsertTableSinkFactory}. */ -public class Elasticsearch6UpsertTableSinkFactoryTest extends ElasticsearchUpsertTableSinkFactoryTestBase { - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - final IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator(INDEX, schema); - - final TestElasticsearch6UpsertTableSink testSink = new TestElasticsearch6UpsertTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions()); - - final DataStreamMock dataStreamMock = new DataStreamMock( - new StreamExecutionEnvironmentMock(), - Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - - testSink.consumeDataStream(dataStreamMock); - - final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( - Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), - new ElasticsearchUpsertSinkFunction( - indexGenerator, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - Elasticsearch6UpsertTableSink.UPDATE_REQUEST_FACTORY, - new int[0])); - expectedBuilder.setFailureHandler(new DummyFailureHandler()); - expectedBuilder.setBulkFlushBackoff(true); - expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - expectedBuilder.setBulkFlushBackoffDelay(123); - expectedBuilder.setBulkFlushBackoffRetries(3); - expectedBuilder.setBulkFlushInterval(100); - expectedBuilder.setBulkFlushMaxActions(1000); - expectedBuilder.setBulkFlushMaxSizeMb(1); - expectedBuilder.setRestClientFactory(new DefaultRestClientFactory(100, "/myapp")); - assertEquals(expectedBuilder, testSink.builder); - } - - @Override - protected String getElasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_6; - } - - @Override - protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator) { - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - private static class TestElasticsearch6UpsertTableSink extends Elasticsearch6UpsertTableSink { - - public ElasticsearchSink.Builder> builder; - - public TestElasticsearch6UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, - List httpHosts) { - builder = super.createBuilder(upsertSinkFunction, httpHosts); - return builder; - } - } - - private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - throw new UnsupportedOperationException(); - } - } - - private static class DataStreamMock extends DataStream> { - - public SinkFunction sinkFunction; - - public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation> outType) { - super(environment, new TransformationMock("name", outType, 1)); - } - - @Override - public DataStreamSink> addSink(SinkFunction> sinkFunction) { - this.sinkFunction = sinkFunction; - return super.addSink(sinkFunction); - } - } - - private static class TransformationMock extends Transformation> { - - public TransformationMock(String name, TypeInformation> outputType, int parallelism) { - super(name, outputType, parallelism); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } +public class Elasticsearch6UpsertTableSinkFactoryTest + extends ElasticsearchUpsertTableSinkFactoryTestBase { + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + final IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator(INDEX, schema); + + final TestElasticsearch6UpsertTableSink testSink = + new TestElasticsearch6UpsertTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + final DataStreamMock dataStreamMock = + new DataStreamMock( + new StreamExecutionEnvironmentMock(), + Types.TUPLE(Types.BOOLEAN, schema.toRowType())); + + testSink.consumeDataStream(dataStreamMock); + + final ElasticsearchSink.Builder> expectedBuilder = + new ElasticsearchSink.Builder<>( + Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), + new ElasticsearchUpsertSinkFunction( + indexGenerator, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + Elasticsearch6UpsertTableSink.UPDATE_REQUEST_FACTORY, + new int[0])); + expectedBuilder.setFailureHandler(new DummyFailureHandler()); + expectedBuilder.setBulkFlushBackoff(true); + expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + expectedBuilder.setBulkFlushBackoffDelay(123); + expectedBuilder.setBulkFlushBackoffRetries(3); + expectedBuilder.setBulkFlushInterval(100); + expectedBuilder.setBulkFlushMaxActions(1000); + expectedBuilder.setBulkFlushMaxSizeMb(1); + expectedBuilder.setRestClientFactory(new DefaultRestClientFactory(100, "/myapp")); + assertEquals(expectedBuilder, testSink.builder); + } + + @Override + protected String getElasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_6; + } + + @Override + protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + IndexGenerator indexGenerator) { + return new Elasticsearch6UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + private static class TestElasticsearch6UpsertTableSink extends Elasticsearch6UpsertTableSink { + + public ElasticsearchSink.Builder> builder; + + public TestElasticsearch6UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { + builder = super.createBuilder(upsertSinkFunction, httpHosts); + return builder; + } + } + + private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { + + @Override + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { + throw new UnsupportedOperationException(); + } + } + + private static class DataStreamMock extends DataStream> { + + public SinkFunction sinkFunction; + + public DataStreamMock( + StreamExecutionEnvironment environment, + TypeInformation> outType) { + super(environment, new TransformationMock("name", outType, 1)); + } + + @Override + public DataStreamSink> addSink( + SinkFunction> sinkFunction) { + this.sinkFunction = sinkFunction; + return super.addSink(sinkFunction); + } + } + + private static class TransformationMock extends Transformation> { + + public TransformationMock( + String name, TypeInformation> outputType, int parallelism) { + super(name, outputType, parallelism); + } + + @Override + public List> getTransitivePredecessors() { + return null; + } + + @Override + public List> getInputs() { + return Collections.emptyList(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 5663aab0..76a70203 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -37,94 +37,100 @@ import java.util.ArrayList; import java.util.List; -/** - * IT cases for the {@link ElasticsearchSink}. - */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName - .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("6.3.1")); - - @Override - protected String getClusterName() { - return "docker-cluster"; - } - - @Override - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = getClusterName(); - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings) - .addTransportAddress(transportAddress); - } - - @Test - public void testElasticsearchSink() throws Exception { - runElasticsearchSinkTest(); - } - - @Test - public void testElasticsearchSinkWithSmile() throws Exception { - runElasticsearchSinkSmileTest(); - } - - @Test - public void testNullAddresses() { - runNullAddressesTest(); - } - - @Test - public void testEmptyAddresses() { - runEmptyAddressesTest(); - } - - @Test - public void testInvalidElasticsearchCluster() throws Exception{ - runInvalidElasticsearchClusterTest(); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - int bulkFlushMaxActions, - String clusterName, - List httpHosts, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { - - ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkFlushMaxActions(bulkFlushMaxActions); - - return builder.build(); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { - - return createElasticsearchSinkForNode( - bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, elasticsearchContainer.getHttpHostAddress()); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction, - String hostAddress) { - - ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(HttpHost.create(hostAddress)); - - ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkFlushMaxActions(bulkFlushMaxActions); - - return builder.build(); - } +/** IT cases for the {@link ElasticsearchSink}. */ +public class ElasticsearchSinkITCase + extends ElasticsearchSinkTestBase { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("6.3.1")); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = getClusterName(); + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + + @Test + public void testNullAddresses() { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception { + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, + clusterName, + elasticsearchSinkFunction, + elasticsearchContainer.getHttpHostAddress()); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String hostAddress) { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(HttpHost.create(hostAddress)); + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java index 0aff5756..733f774a 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -29,43 +29,42 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; -/** - * Elasticsearch 7 specific configuration. - */ +/** Elasticsearch 7 specific configuration. */ @Internal final class Elasticsearch7Configuration extends ElasticsearchConfiguration { - Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { - super(config, classLoader); - } + Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } - public List getHosts() { - return config.get(HOSTS_OPTION).stream() - .map(Elasticsearch7Configuration::validateAndParseHostsString) - .collect(Collectors.toList()); - } + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch7Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } - private static HttpHost validateAndParseHostsString(String host) { - try { - HttpHost httpHost = HttpHost.create(host); - if (httpHost.getPort() < 0) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", - host, - HOSTS_OPTION.key())); - } + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } - if (httpHost.getSchemeName() == null) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", - host, - HOSTS_OPTION.key())); - } - return httpHost; - } catch (Exception e) { - throw new ValidationException(String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", - host, - HOSTS_OPTION.key()), e); - } - } + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 52efbe1c..0fe2dac4 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -49,267 +49,270 @@ import java.util.Objects; /** - * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a logical - * description. + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. */ @Internal final class Elasticsearch7DynamicSink implements DynamicTableSink { - @VisibleForTesting - static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7DynamicSink.Elasticsearch7RequestFactory(); - - private final EncodingFormat> format; - private final TableSchema schema; - private final Elasticsearch7Configuration config; - - public Elasticsearch7DynamicSink( - EncodingFormat> format, - Elasticsearch7Configuration config, - TableSchema schema) { - this(format, config, schema, (ElasticsearchSink.Builder::new)); - } - - //-------------------------------------------------------------- - // Hack to make configuration testing possible. - // - // The code in this block should never be used outside of tests. - // Having a way to inject a builder we can assert the builder in - // the test. We can not assert everything though, e.g. it is not - // possible to assert flushing on checkpoint, as it is configured - // on the sink itself. - //-------------------------------------------------------------- - - private final ElasticSearchBuilderProvider builderProvider; - - @FunctionalInterface - interface ElasticSearchBuilderProvider { - ElasticsearchSink.Builder createBuilder( - List httpHosts, - RowElasticsearchSinkFunction upsertSinkFunction); - } - - Elasticsearch7DynamicSink( - EncodingFormat> format, - Elasticsearch7Configuration config, - TableSchema schema, - ElasticSearchBuilderProvider builderProvider) { - this.format = format; - this.schema = schema; - this.config = config; - this.builderProvider = builderProvider; - } - - //-------------------------------------------------------------- - // End of hack to make configuration testing possible - //-------------------------------------------------------------- - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - if (kind != RowKind.UPDATE_BEFORE) { - builder.addContainedKind(kind); - } - } - return builder.build(); - } - - @Override - public SinkFunctionProvider getSinkRuntimeProvider(Context context) { - return () -> { - SerializationSchema format = this.format.createRuntimeEncoder(context, schema.toRowDataType()); - - final RowElasticsearchSinkFunction upsertFunction = - new RowElasticsearchSinkFunction( - IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), - null, // this is deprecated in es 7+ - format, - XContentType.JSON, - REQUEST_FACTORY, - KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter()) - ); - - final ElasticsearchSink.Builder builder = builderProvider.createBuilder( - config.getHosts(), - upsertFunction); - - builder.setFailureHandler(config.getFailureHandler()); - builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); - builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); - builder.setBulkFlushInterval(config.getBulkFlushInterval()); - builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); - config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); - config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); - config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); - - // we must overwrite the default factory which is defined with a lambda because of a bug - // in shading lambda serialization shading see FLINK-18006 - if (config.getUsername().isPresent() - && config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { - builder.setRestClientFactory(new AuthRestClientFactory(config.getPathPrefix().orElse(null), config.getUsername().get(), config.getPassword().get())); - } else { - builder.setRestClientFactory(new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); - } - - final ElasticsearchSink sink = builder.build(); - - if (config.isDisableFlushOnCheckpoint()) { - sink.disableFlushOnCheckpoint(); - } - - return sink; - }; - } - - @Override - public DynamicTableSink copy() { - return this; - } - - @Override - public String asSummaryString() { - return "Elasticsearch7"; - } - - /** - * Serializable {@link RestClientFactory} used by the sink. - */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** - * Serializable {@link RestClientFactory} used by the sink which enable authentication. - */ - @VisibleForTesting - static class AuthRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - private final String username; - private final String password; - private transient CredentialsProvider credentialsProvider; - - public AuthRestClientFactory(@Nullable String pathPrefix, String username, String password) { - this.pathPrefix = pathPrefix; - this.password = password; - this.username = username; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - if (credentialsProvider == null) { - credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(username, password)); - } - restClientBuilder.setHttpClientConfigCallback(httpAsyncClientBuilder -> - httpAsyncClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AuthRestClientFactory that = (AuthRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix) && - Objects.equals(username, that.username) && - Objects.equals(password, that.password); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix, password, username); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. - */ - private static class Elasticsearch7RequestFactory implements RequestFactory { - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new IndexRequest(index) - .id(key) - .source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, key); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; - return Objects.equals(format, that.format) && - Objects.equals(schema, that.schema) && - Objects.equals(config, that.config) && - Objects.equals(builderProvider, that.builderProvider); - } - - @Override - public int hashCode() { - return Objects.hash(format, schema, config, builderProvider); - } + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = + new Elasticsearch7DynamicSink.Elasticsearch7RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch7Configuration config; + + public Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema) { + this(format, config, schema, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), + null, // this is deprecated in es 7+ + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, password, username); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index).id(key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index a68a0efa..96344194 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -55,119 +55,116 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; -/** - * A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. - */ +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ @Internal public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { - private static final Set> requiredOptions = Stream.of( - HOSTS_OPTION, - INDEX_OPTION - ).collect(Collectors.toSet()); - private static final Set> optionalOptions = Stream.of( - KEY_DELIMITER_OPTION, - FAILURE_HANDLER_OPTION, - FLUSH_ON_CHECKPOINT_OPTION, - BULK_FLASH_MAX_SIZE_OPTION, - BULK_FLUSH_MAX_ACTIONS_OPTION, - BULK_FLUSH_INTERVAL_OPTION, - BULK_FLUSH_BACKOFF_TYPE_OPTION, - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, - BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, - CONNECTION_PATH_PREFIX, - FORMAT_OPTION, - PASSWORD_OPTION, - USERNAME_OPTION - ).collect(Collectors.toSet()); - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - TableSchema tableSchema = context.getCatalogTable().getSchema(); - ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); - - final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); - - final EncodingFormat> format = helper.discoverEncodingFormat( - SerializationFormatFactory.class, - FORMAT_OPTION); - - helper.validate(); - Configuration configuration = new Configuration(); - context.getCatalogTable() - .getOptions() - .forEach(configuration::setString); - Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration, context.getClassLoader()); - - validate(config, configuration); - - return new Elasticsearch7DynamicSink( - format, - config, - TableSchemaUtils.getPhysicalSchema(tableSchema)); - } - - private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { - config.getFailureHandler(); // checks if we can instantiate the custom failure handler - config.getHosts(); // validate hosts - validate( - config.getIndex().length() >= 1, - () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); - int maxActions = config.getBulkFlushMaxActions(); - validate( - maxActions == -1 || maxActions >= 1, - () -> String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - maxActions) - ); - long maxSize = config.getBulkFlushMaxByteSize(); - long mb1 = 1024 * 1024; - validate( - maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), - () -> String.format( - "'%s' must be in MB granularity. Got: %s", - BULK_FLASH_MAX_SIZE_OPTION.key(), - originalConfiguration.get(BULK_FLASH_MAX_SIZE_OPTION).toHumanReadableString()) - ); - validate( - config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), - () -> String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), - config.getBulkFlushBackoffRetries().get()) - ); - if (config.getUsername().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { - validate( - config.getPassword().isPresent() && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), - () -> String.format( - "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", - USERNAME_OPTION.key(), - PASSWORD_OPTION.key(), - config.getUsername().get(), - config.getPassword().orElse("") - )); - } - } - - private static void validate(boolean condition, Supplier message) { - if (!condition) { - throw new ValidationException(message.get()); - } - } - - @Override - public String factoryIdentifier() { - return "elasticsearch-7"; - } - - @Override - public Set> requiredOptions() { - return requiredOptions; - } - - @Override - public Set> optionalOptions() { - return optionalOptions; - } + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch7Configuration config = + new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch7DynamicSink( + format, config, TableSchemaUtils.getPhysicalSchema(tableSchema)); + } + + private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-7"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index 8890eca1..6dff0559 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -42,105 +42,105 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; -/** - * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. - */ +/** Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 7 and later versions. */ @Internal -public class Elasticsearch7ApiCallBridge implements ElasticsearchApiCallBridge { - - private static final long serialVersionUID = -5222683870097809633L; - - private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class); - - /** - * User-provided HTTP Host. - */ - private final List httpHosts; - - /** - * The factory to configure the rest client. - */ - private final RestClientFactory restClientFactory; - - Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { - Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); - this.httpHosts = httpHosts; - this.restClientFactory = Preconditions.checkNotNull(restClientFactory); - } - - @Override - public RestHighLevelClient createClient(Map clientConfig) { - RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); - restClientFactory.configureRestClientBuilder(builder); - - RestHighLevelClient rhlClient = new RestHighLevelClient(builder); - - return rhlClient; - } - - @Override - public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { - return BulkProcessor.builder((request, bulkListener) -> client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener); - } - - @Override - public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { - if (!bulkItemResponse.isFailed()) { - return null; - } else { - return bulkItemResponse.getFailure().getCause(); - } - } - - @Override - public void configureBulkProcessorBackoff( - BulkProcessor.Builder builder, - @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { - - BackoffPolicy backoffPolicy; - if (flushBackoffPolicy != null) { - switch (flushBackoffPolicy.getBackoffType()) { - case CONSTANT: - backoffPolicy = BackoffPolicy.constantBackoff( - new TimeValue(flushBackoffPolicy.getDelayMillis()), - flushBackoffPolicy.getMaxRetryCount()); - break; - case EXPONENTIAL: - default: - backoffPolicy = BackoffPolicy.exponentialBackoff( - new TimeValue(flushBackoffPolicy.getDelayMillis()), - flushBackoffPolicy.getMaxRetryCount()); - } - } else { - backoffPolicy = BackoffPolicy.noBackoff(); - } - - builder.setBackoffPolicy(backoffPolicy); - } - - @Override - public RequestIndexer createBulkProcessorIndexer( - BulkProcessor bulkProcessor, - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new Elasticsearch7BulkProcessorIndexer( - bulkProcessor, - flushOnCheckpoint, - numPendingRequestsRef); - } - - @Override - public void verifyClientConnection(RestHighLevelClient client) throws IOException { - if (LOG.isInfoEnabled()) { - LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); - } - - if (!client.ping(RequestOptions.DEFAULT)) { - throw new RuntimeException("There are no reachable Elasticsearch nodes!"); - } - - if (LOG.isInfoEnabled()) { - LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); - } - } +public class Elasticsearch7ApiCallBridge + implements ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch7ApiCallBridge.class); + + /** User-provided HTTP Host. */ + private final List httpHosts; + + /** The factory to configure the rest client. */ + private final RestClientFactory restClientFactory; + + Elasticsearch7ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + @Override + public RestHighLevelClient createClient(Map clientConfig) { + RestClientBuilder builder = + RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder( + RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder( + (request, bulkListener) -> + client.bulkAsync(request, RequestOptions.DEFAULT, bulkListener), + listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = + BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = + BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new Elasticsearch7BulkProcessorIndexer( + bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } + + @Override + public void verifyClientConnection(RestHighLevelClient client) throws IOException { + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + + if (!client.ping(RequestOptions.DEFAULT)) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + + if (LOG.isInfoEnabled()) { + LOG.info("Elasticsearch RestHighLevelClient is connected to {}", httpHosts.toString()); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java index 0b7a4374..866c0586 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java @@ -32,54 +32,54 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. - * {@link ActionRequest ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. + * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest + * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. * *

Note: This class is binary compatible to Elasticsearch 7. */ @Internal class Elasticsearch7BulkProcessorIndexer implements RequestIndexer { - private final BulkProcessor bulkProcessor; - private final boolean flushOnCheckpoint; - private final AtomicLong numPendingRequestsRef; + private final BulkProcessor bulkProcessor; + private final boolean flushOnCheckpoint; + private final AtomicLong numPendingRequestsRef; - Elasticsearch7BulkProcessorIndexer( - BulkProcessor bulkProcessor, - boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - this.bulkProcessor = checkNotNull(bulkProcessor); - this.flushOnCheckpoint = flushOnCheckpoint; - this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); - } + Elasticsearch7BulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + this.bulkProcessor = checkNotNull(bulkProcessor); + this.flushOnCheckpoint = flushOnCheckpoint; + this.numPendingRequestsRef = checkNotNull(numPendingRequestsRef); + } - @Override - public void add(DeleteRequest... deleteRequests) { - for (DeleteRequest deleteRequest : deleteRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(deleteRequest); - } - } + @Override + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(deleteRequest); + } + } - @Override - public void add(IndexRequest... indexRequests) { - for (IndexRequest indexRequest : indexRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(indexRequest); - } - } + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } - @Override - public void add(UpdateRequest... updateRequests) { - for (UpdateRequest updateRequest : updateRequests) { - if (flushOnCheckpoint) { - numPendingRequestsRef.getAndIncrement(); - } - this.bulkProcessor.add(updateRequest); - } - } + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java index 7795fb8e..06c2d49f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java @@ -55,229 +55,226 @@ import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.DISABLE_FLUSH_ON_CHECKPOINT; import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; -/** - * Version-specific upsert table sink for Elasticsearch 7. - */ +/** Version-specific upsert table sink for Elasticsearch 7. */ @Internal public class Elasticsearch7UpsertTableSink extends ElasticsearchUpsertTableSinkBase { - @VisibleForTesting - static final RequestFactory UPDATE_REQUEST_FACTORY = - new Elasticsearch7RequestFactory(); - - public Elasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - "", - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @VisibleForTesting - Elasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @Override - protected ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertSinkFunction) { - - final List httpHosts = hosts.stream() - .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) - .collect(Collectors.toList()); - - final ElasticsearchSink.Builder> builder = createBuilder(upsertSinkFunction, httpHosts); - - builder.setFailureHandler(failureHandler); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) - .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) - .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) - .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) - .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) - .ifPresent(v -> builder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) - .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) - .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); - - builder.setRestClientFactory( - new DefaultRestClientFactory(sinkOptions.get(REST_PATH_PREFIX))); - - final ElasticsearchSink> sink = builder.build(); - - Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) - .ifPresent(v -> { - if (Boolean.valueOf(v)) { - sink.disableFlushOnCheckpoint(); - } - }); - - return sink; - } - - @VisibleForTesting - ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, - List httpHosts) { - return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** - * Serializable {@link RestClientFactory} used by the sink. - */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the sink. - */ - private static class Elasticsearch7RequestFactory implements RequestFactory { - - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - XContentType contentType, - byte[] document) { - return new IndexRequest(index) - .source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, key); - } - } + @VisibleForTesting + static final RequestFactory UPDATE_REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + + public Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + "", + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @VisibleForTesting + Elasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions, + UPDATE_REQUEST_FACTORY); + } + + @Override + protected ElasticsearchUpsertTableSinkBase copy( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + RequestFactory requestFactory) { + + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected SinkFunction> createSinkFunction( + List hosts, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + ElasticsearchUpsertSinkFunction upsertSinkFunction) { + + final List httpHosts = + hosts.stream() + .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) + .collect(Collectors.toList()); + + final ElasticsearchSink.Builder> builder = + createBuilder(upsertSinkFunction, httpHosts); + + builder.setFailureHandler(failureHandler); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) + .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) + .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) + .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) + .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) + .ifPresent( + v -> + builder.setBulkFlushBackoffType( + ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) + .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); + + Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) + .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); + + builder.setRestClientFactory( + new DefaultRestClientFactory(sinkOptions.get(REST_PATH_PREFIX))); + + final ElasticsearchSink> sink = builder.build(); + + Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) + .ifPresent( + v -> { + if (Boolean.valueOf(v)) { + sink.disableFlushOnCheckpoint(); + } + }); + + return sink; + } + + @VisibleForTesting + ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { + return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, String docType, XContentType contentType, byte[] document) { + return new IndexRequest(index).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java index 29208c80..6cded3c2 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java @@ -36,41 +36,39 @@ import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; -/** - * Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 7. - */ +/** Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 7. */ @Internal public class Elasticsearch7UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { - @Override - protected String elasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_7; - } + @Override + protected String elasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } - @Override - protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { + @Override + protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java index 5aae8954..3ac23435 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -35,204 +35,225 @@ import java.util.Objects; /** - * Elasticsearch 7.x sink that requests multiple {@link ActionRequest ActionRequests} - * against a cluster for each incoming element. + * Elasticsearch 7.x sink that requests multiple {@link ActionRequest ActionRequests} against a + * cluster for each incoming element. * - *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch cluster. - * The sink will fail if no cluster can be connected to using the provided transport addresses passed to the constructor. + *

The sink internally uses a {@link RestHighLevelClient} to communicate with an Elasticsearch + * cluster. The sink will fail if no cluster can be connected to using the provided transport + * addresses passed to the constructor. + * + *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest + * ActionRequests}. This will buffer elements before sending a request to the cluster. The behaviour + * of the {@code BulkProcessor} can be configured using these config keys: * - *

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. - * This will buffer elements before sending a request to the cluster. The behaviour of the - * {@code BulkProcessor} can be configured using these config keys: *

    - *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer - *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer - *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two - * settings in milliseconds + *
  • {@code bulk.flush.max.actions}: Maximum amount of elements to buffer + *
  • {@code bulk.flush.max.size.mb}: Maximum amount of data (in megabytes) to buffer + *
  • {@code bulk.flush.interval.ms}: Interval at which to flush data regardless of the other two + * settings in milliseconds *
* *

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple - * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of - * {@link ElasticsearchSinkFunction} for an example. + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation + * of {@link ElasticsearchSinkFunction} for an example. * * @param Type of the elements handled by this sink */ @PublicEvolving public class ElasticsearchSink extends ElasticsearchSinkBase { - private static final long serialVersionUID = 1L; - - private ElasticsearchSink( - Map bulkRequestsConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler, - RestClientFactory restClientFactory) { - - super(new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); - } - - /** - * A builder for creating an {@link ElasticsearchSink}. - * - * @param Type of the elements handled by the sink this builder creates. - */ - @PublicEvolving - public static class Builder { - - private final List httpHosts; - private final ElasticsearchSinkFunction elasticsearchSinkFunction; - - private Map bulkRequestsConfig = new HashMap<>(); - private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); - private RestClientFactory restClientFactory = restClientBuilder -> {}; - - /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. - * - * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. - */ - public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { - this.httpHosts = Preconditions.checkNotNull(httpHosts); - this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); - } - - /** - * Sets the maximum number of actions to buffer for each bulk request. - * - * @param numMaxActions the maximum number of actions to buffer per bulk request. - */ - public void setBulkFlushMaxActions(int numMaxActions) { - Preconditions.checkArgument( - numMaxActions > 0, - "Max number of buffered actions must be larger than 0."); - - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); - } - - /** - * Sets the maximum size of buffered actions, in mb, per bulk request. - * - * @param maxSizeMb the maximum size of buffered actions, in mb. - */ - public void setBulkFlushMaxSizeMb(int maxSizeMb) { - Preconditions.checkArgument( - maxSizeMb > 0, - "Max size of buffered actions must be larger than 0."); - - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); - } - - /** - * Sets the bulk flush interval, in milliseconds. - * - * @param intervalMillis the bulk flush interval, in milliseconds. - */ - public void setBulkFlushInterval(long intervalMillis) { - Preconditions.checkArgument( - intervalMillis >= 0, - "Interval (in milliseconds) between each flush must be larger than or equal to 0."); - - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); - } - - /** - * Sets whether or not to enable bulk flush backoff behaviour. - * - * @param enabled whether or not to enable backoffs. - */ - public void setBulkFlushBackoff(boolean enabled) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); - } - - /** - * Sets the type of back of to use when flushing bulk requests. - * - * @param flushBackoffType the backoff type to use. - */ - public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { - this.bulkRequestsConfig.put( - CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, - Preconditions.checkNotNull(flushBackoffType).toString()); - } - - /** - * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. - * - * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests - */ - public void setBulkFlushBackoffRetries(int maxRetries) { - Preconditions.checkArgument( - maxRetries > 0, - "Max number of backoff attempts must be larger than 0."); - - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); - } - - /** - * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. - * - * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. - */ - public void setBulkFlushBackoffDelay(long delayMillis) { - Preconditions.checkArgument( - delayMillis >= 0, - "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); - } - - /** - * Sets a failure handler for action requests. - * - * @param failureHandler This is used to handle failed {@link ActionRequest}. - */ - public void setFailureHandler(ActionRequestFailureHandler failureHandler) { - this.failureHandler = Preconditions.checkNotNull(failureHandler); - } - - /** - * Sets a REST client factory for custom client configuration. - * - * @param restClientFactory the factory that configures the rest client. - */ - public void setRestClientFactory(RestClientFactory restClientFactory) { - this.restClientFactory = Preconditions.checkNotNull(restClientFactory); - } - - /** - * Creates the Elasticsearch sink. - * - * @return the created Elasticsearch sink. - */ - public ElasticsearchSink build() { - return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Builder builder = (Builder) o; - return Objects.equals(httpHosts, builder.httpHosts) && - Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) && - Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) && - Objects.equals(failureHandler, builder.failureHandler) && - Objects.equals(restClientFactory, builder.restClientFactory); - } - - @Override - public int hashCode() { - return Objects.hash( - httpHosts, - elasticsearchSinkFunction, - bulkRequestsConfig, - failureHandler, - restClientFactory); - } - } + private static final long serialVersionUID = 1L; + + private ElasticsearchSink( + Map bulkRequestsConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { + + super( + new Elasticsearch7ApiCallBridge(httpHosts, restClientFactory), + bulkRequestsConfig, + elasticsearchSinkFunction, + failureHandler); + } + + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + */ + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link + * RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} + * connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} + * from the incoming element. + */ + public Builder( + List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions > 0, "Max number of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb > 0, "Max size of buffered actions must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk + * requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk + * requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); + } + + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = Preconditions.checkNotNull(failureHandler); + } + + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); + } + + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ + public ElasticsearchSink build() { + return new ElasticsearchSink<>( + bulkRequestsConfig, + httpHosts, + elasticsearchSinkFunction, + failureHandler, + restClientFactory); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Builder builder = (Builder) o; + return Objects.equals(httpHosts, builder.httpHosts) + && Objects.equals(elasticsearchSinkFunction, builder.elasticsearchSinkFunction) + && Objects.equals(bulkRequestsConfig, builder.bulkRequestsConfig) + && Objects.equals(failureHandler, builder.failureHandler) + && Objects.equals(restClientFactory, builder.restClientFactory); + } + + @Override + public int hashCode() { + return Objects.hash( + httpHosts, + elasticsearchSinkFunction, + bulkRequestsConfig, + failureHandler, + restClientFactory); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java index 6001f43c..50e2dfc1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java @@ -24,17 +24,16 @@ import java.io.Serializable; /** - * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally - * used in the {@link ElasticsearchSink}. + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} + * internally used in the {@link ElasticsearchSink}. */ @PublicEvolving public interface RestClientFactory extends Serializable { - /** - * Configures the rest client builder. - * - * @param restClientBuilder the configured rest client builder. - */ - void configureRestClientBuilder(RestClientBuilder restClientBuilder); - + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index d34e819c..14aef72f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -29,192 +29,187 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; -/** - * Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. - */ +/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ public class Elasticsearch7DynamicSinkFactoryTest { - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Test - public void validateEmptyConfiguration() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "One or more required options are missing.\n" + - "\n" + - "Missing required options are:\n" + - "\n" + - "hosts\n" + - "index"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .build() - ); - } - - @Test - public void validateWrongIndex() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'index' must not be empty"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption("index", "") - .withOption("hosts", "http://localhost:12345") - .build() - ); - } - - @Test - public void validateWrongHosts() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption("index", "MyIndex") - .withOption("hosts", "wrong-host") - .build() - ); - } - - @Test - public void validateWrongFlushSize() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") - .build() - ); - } - - @Test - public void validateWrongRetries() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "0") - .build() - ); - } - - @Test - public void validateWrongMaxActions() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") - .build() - ); - } - - @Test - public void validateWrongBackoffDelay() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Invalid value for option 'sink.bulk-flush.backoff.delay'."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") - .build() - ); - } - - @Test - public void validatePrimaryKeyOnIllegalColumn() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "The table has a primary key on columns of illegal types: " + - "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + - " Elasticsearch sink does not support primary keys on columns of types: " + - "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.ARRAY(DataTypes.BIGINT().notNull()).notNull()) - .field("c", DataTypes.MAP(DataTypes.BIGINT(), DataTypes.STRING()).notNull()) - .field("d", DataTypes.MULTISET(DataTypes.BIGINT().notNull()).notNull()) - .field("e", DataTypes.ROW(DataTypes.FIELD("a", DataTypes.BIGINT())).notNull()) - .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) - .field("g", DataTypes.BYTES().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g") - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") - .build() - ); - } - - @Test - public void validateWrongCredential() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder() - .field("a", DataTypes.TIME()) - .build()) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") - .build() - ); - } + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption("index", "") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption("index", "MyIndex") + .withOption("hosts", "wrong-host") + .build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field( + "b", + DataTypes.ARRAY(DataTypes.BIGINT().notNull()) + .notNull()) + .field( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()) + .field( + "d", + DataTypes.MULTISET(DataTypes.BIGINT().notNull()) + .notNull()) + .field( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", DataTypes.BIGINT())) + .notNull()) + .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field("g", DataTypes.BYTES().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g") + .build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) + .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + .build()); + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index c8bb4e8c..b92f34e8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -59,264 +59,308 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -/** - * IT tests for {@link Elasticsearch7DynamicSink}. - */ +/** IT tests for {@link Elasticsearch7DynamicSink}. */ public class Elasticsearch7DynamicSinkITCase { - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName - .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("7.5.1")); + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("7.5.1")); - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings) - .addTransportAddress(transportAddress); - } + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } - @Test - public void testWritingDocuments() throws Exception { - TableSchema schema = TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.TIME()) - .field("c", DataTypes.STRING().notNull()) - .field("d", DataTypes.FLOAT()) - .field("e", DataTypes.TINYINT().notNull()) - .field("f", DataTypes.DATE()) - .field("g", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "g") - .build(); - GenericRowData rowData = GenericRowData.of( - 1L, - 12345, - StringData.fromString("ABCDE"), - 12.12f, - (byte) 2, - 12345, - TimestampData.fromLocalDateTime(LocalDateTime.parse("2012-12-12T12:12:12"))); + @Test + public void testWritingDocuments() throws Exception { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.TIME()) + .field("c", DataTypes.STRING().notNull()) + .field("d", DataTypes.FLOAT()) + .field("e", DataTypes.TINYINT().notNull()) + .field("f", DataTypes.DATE()) + .field("g", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "g") + .build(); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); - String index = "writing-documents"; - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + String index = "writing-documents"; + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - SinkFunctionProvider sinkRuntimeProvider = (SinkFunctionProvider) sinkFactory.createDynamicTableSink( - context() - .withSchema(schema) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), index) - .withOption(ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) - .withOption(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") - .build() - ).getSinkRuntimeProvider(new MockContext()); + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchOptions.INDEX_OPTION.key(), + index) + .withOption( + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); - SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); - StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); - rowData.setRowKind(RowKind.UPDATE_AFTER); - environment.fromElements(rowData).addSink(sinkFunction); - environment.execute(); + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); - Client client = getClient(); - Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - @Test - public void testWritingDocumentsFromTableApi() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "table-api"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIME,\n" + - "c STRING NOT NULL,\n" + - "d FLOAT,\n" + - "e TINYINT NOT NULL,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL," + - "h as a + 2,\n" + - "PRIMARY KEY (a, g) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL," + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")) - ).executeInsert("esTable").await(); + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); - Map response = client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - @Test - public void testWritingDocumentsNoPrimaryKey() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "no-primary-key"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIME,\n" + - "c STRING NOT NULL,\n" + - "d FLOAT,\n" + - "e TINYINT NOT NULL,\n" + - "f DATE,\n" + - "g TIMESTAMP NOT NULL\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 2L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "FGHIJK", - 13.13f, - (byte) 4, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2013-12-12T13:13:13")) - ).executeInsert("esTable").await(); + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); + Client client = getClient(); - // search API does not return documents that were not indexed, we might need to query - // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); - SearchHits hits; - do { - hits = client.prepareSearch(index) - .execute() - .actionGet() - .getHits(); - if (hits.getTotalHits().value < 2) { - Thread.sleep(200); - } - } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits().value < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); - if (hits.getTotalHits().value < 2) { - throw new AssertionError("Could not retrieve results from Elasticsearch."); - } + if (hits.getTotalHits().value < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } - HashSet> resultSet = new HashSet<>(); - resultSet.add(hits.getAt(0).getSourceAsMap()); - resultSet.add(hits.getAt(1).getSourceAsMap()); - Map expectedMap1 = new HashMap<>(); - expectedMap1.put("a", 1); - expectedMap1.put("b", "00:00:12"); - expectedMap1.put("c", "ABCDE"); - expectedMap1.put("d", 12.12d); - expectedMap1.put("e", 2); - expectedMap1.put("f", "2003-10-20"); - expectedMap1.put("g", "2012-12-12 12:12:12"); - Map expectedMap2 = new HashMap<>(); - expectedMap2.put("a", 2); - expectedMap2.put("b", "00:00:12"); - expectedMap2.put("c", "FGHIJK"); - expectedMap2.put("d", 13.13d); - expectedMap2.put("e", 4); - expectedMap2.put("f", "2003-10-20"); - expectedMap2.put("g", "2013-12-12 13:13:13"); - HashSet> expectedSet = new HashSet<>(); - expectedSet.add(expectedMap1); - expectedSet.add(expectedMap2); - assertThat(resultSet, equalTo(expectedSet)); - } + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } - @Test - public void testWritingDocumentsWithDynamicIndex() throws Exception { - TableEnvironment tableEnvironment = TableEnvironment.create(EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create( + EnvironmentSettings.newInstance() + .useBlinkPlanner() + .inStreamingMode() + .build()); - String index = "dynamic-index-{b|yyyy-MM-dd}"; - tableEnvironment.executeSql("CREATE TABLE esTable (" + - "a BIGINT NOT NULL,\n" + - "b TIMESTAMP NOT NULL,\n" + - "PRIMARY KEY (a) NOT ENFORCED\n" + - ")\n" + - "WITH (\n" + - String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + - String.format("'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + - String.format("'%s'='%s',\n", ElasticsearchOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + - String.format("'%s'='%s'\n", ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + - ")"); + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + + ")"); - tableEnvironment.fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable").await(); + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); - Client client = getClient(); - Map response = client.get(new GetRequest("dynamic-index-2012-12-12", "1")) - .actionGet() - .getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); - } + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", "1")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } - private static class MockContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { - return null; - } - } + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index 7e18d64c..3f3b640d 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -45,211 +45,234 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; -/** - * Tests for {@link Elasticsearch7DynamicSink} parameters. - */ +/** Tests for {@link Elasticsearch7DynamicSink} parameters. */ public class Elasticsearch7DynamicSinkTest { - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - private static final String HOSTNAME = "host1"; - private static final int PORT = 1234; - private static final String SCHEMA = "https"; - private static final String INDEX = "MyIndex"; - private static final String DOC_TYPE = "MyType"; - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration(getConfig(), this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(true); - verify(provider.builderSpy).setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - verify(provider.builderSpy).setBulkFlushBackoffDelay(123); - verify(provider.builderSpy).setBulkFlushBackoffRetries(3); - verify(provider.builderSpy).setBulkFlushInterval(100); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); - verify(provider.sinkSpy).disableFlushOnCheckpoint(); - } - - @Test - public void testDefaultConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration(configuration, this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - @Test - public void testAuthConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration(configuration, this.getClass().getClassLoader()), - schema, - provider - ); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy).setRestClientFactory(new Elasticsearch7DynamicSink.AuthRestClientFactory(null, USERNAME, PASSWORD)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - private Configuration getConfig() { - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString(ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); - configuration.setString(ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); - configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); - return configuration; - } - - private static class BuilderProvider implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { - public ElasticsearchSink.Builder builderSpy; - public ElasticsearchSink sinkSpy; - - @Override - public ElasticsearchSink.Builder createBuilder( - List httpHosts, - RowElasticsearchSinkFunction upsertSinkFunction) { - builderSpy = Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); - doAnswer( - invocation -> { - sinkSpy = Mockito.spy((ElasticsearchSink) invocation.callRealMethod()); - return sinkSpy; - } - ).when(builderSpy).build(); - - return builderSpy; - } - } - - private TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - private static class DummySerializationSchema implements SerializationSchema { - - private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); - - @Override - public byte[] serialize(RowData element) { - return new byte[0]; - } - } - - private static class DummyEncodingFormat implements EncodingFormat> { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, - DataType consumedDataType) { - return DummySerializationSchema.INSTANCE; - } - - @Override - public ChangelogMode getChangelogMode() { - return null; - } - } - - private static class MockSinkContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter(DataType consumedDataType) { - return null; - } - } - - /** - * Custom failure handler for testing. - */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure(ActionRequest action, Throwable failure, int restStatusCode, RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java index 099d6ef9..7b9f4801 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java @@ -54,170 +54,184 @@ import static org.junit.Assert.assertEquals; /** - * Test for {@link Elasticsearch7UpsertTableSink} created by {@link Elasticsearch7UpsertTableSinkFactory}. + * Test for {@link Elasticsearch7UpsertTableSink} created by {@link + * Elasticsearch7UpsertTableSinkFactory}. */ -public class Elasticsearch7UpsertTableSinkFactoryTest extends ElasticsearchUpsertTableSinkFactoryTestBase { - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - final IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator(INDEX, schema); - - final TestElasticsearch7UpsertTableSink testSink = new TestElasticsearch7UpsertTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions()); - - final DataStreamMock dataStreamMock = new DataStreamMock( - new StreamExecutionEnvironmentMock(), - Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - - testSink.consumeDataStream(dataStreamMock); - - final ElasticsearchSink.Builder> expectedBuilder = new ElasticsearchSink.Builder<>( - Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), - new ElasticsearchUpsertSinkFunction( - indexGenerator, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder().withTypeInfo(schema.toRowType()).build(), - XContentType.JSON, - Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, - new int[0])); - expectedBuilder.setFailureHandler(new DummyFailureHandler()); - expectedBuilder.setBulkFlushBackoff(true); - expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - expectedBuilder.setBulkFlushBackoffDelay(123); - expectedBuilder.setBulkFlushBackoffRetries(3); - expectedBuilder.setBulkFlushInterval(100); - expectedBuilder.setBulkFlushMaxActions(1000); - expectedBuilder.setBulkFlushMaxSizeMb(1); - expectedBuilder.setRestClientFactory(new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); - assertEquals(expectedBuilder, testSink.builder); - } - - @Override - protected String getElasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_7; - } - - @Override - protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator) { - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - private static class TestElasticsearch7UpsertTableSink extends Elasticsearch7UpsertTableSink { - - public ElasticsearchSink.Builder> builder; - - public TestElasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, - List httpHosts) { - builder = super.createBuilder(upsertSinkFunction, httpHosts); - return builder; - } - } - - private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - throw new UnsupportedOperationException(); - } - } - - private static class DataStreamMock extends DataStream> { - - public SinkFunction sinkFunction; - - public DataStreamMock(StreamExecutionEnvironment environment, TypeInformation> outType) { - super(environment, new TransformationMock("name", outType, 1)); - } - - @Override - public DataStreamSink> addSink(SinkFunction> sinkFunction) { - this.sinkFunction = sinkFunction; - return super.addSink(sinkFunction); - } - } - - private static class TransformationMock extends Transformation> { - - public TransformationMock(String name, TypeInformation> outputType, int parallelism) { - super(name, outputType, parallelism); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } +public class Elasticsearch7UpsertTableSinkFactoryTest + extends ElasticsearchUpsertTableSinkFactoryTestBase { + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + final IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator(INDEX, schema); + + final TestElasticsearch7UpsertTableSink testSink = + new TestElasticsearch7UpsertTableSink( + false, + schema, + Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), + INDEX, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + new DummyFailureHandler(), + createTestSinkOptions()); + + final DataStreamMock dataStreamMock = + new DataStreamMock( + new StreamExecutionEnvironmentMock(), + Types.TUPLE(Types.BOOLEAN, schema.toRowType())); + + testSink.consumeDataStream(dataStreamMock); + + final ElasticsearchSink.Builder> expectedBuilder = + new ElasticsearchSink.Builder<>( + Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), + new ElasticsearchUpsertSinkFunction( + indexGenerator, + DOC_TYPE, + KEY_DELIMITER, + KEY_NULL_LITERAL, + JsonRowSerializationSchema.builder() + .withTypeInfo(schema.toRowType()) + .build(), + XContentType.JSON, + Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, + new int[0])); + expectedBuilder.setFailureHandler(new DummyFailureHandler()); + expectedBuilder.setBulkFlushBackoff(true); + expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + expectedBuilder.setBulkFlushBackoffDelay(123); + expectedBuilder.setBulkFlushBackoffRetries(3); + expectedBuilder.setBulkFlushInterval(100); + expectedBuilder.setBulkFlushMaxActions(1000); + expectedBuilder.setBulkFlushMaxSizeMb(1); + expectedBuilder.setRestClientFactory( + new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); + assertEquals(expectedBuilder, testSink.builder); + } + + @Override + protected String getElasticsearchVersion() { + return CONNECTOR_VERSION_VALUE_7; + } + + @Override + protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions, + IndexGenerator indexGenerator) { + return new Elasticsearch7UpsertTableSink( + isAppendOnly, + schema, + hosts, + index, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + // -------------------------------------------------------------------------------------------- + // Helper classes + // -------------------------------------------------------------------------------------------- + + private static class TestElasticsearch7UpsertTableSink extends Elasticsearch7UpsertTableSink { + + public ElasticsearchSink.Builder> builder; + + public TestElasticsearch7UpsertTableSink( + boolean isAppendOnly, + TableSchema schema, + List hosts, + String index, + String docType, + String keyDelimiter, + String keyNullLiteral, + SerializationSchema serializationSchema, + XContentType contentType, + ActionRequestFailureHandler failureHandler, + Map sinkOptions) { + + super( + isAppendOnly, + schema, + hosts, + index, + docType, + keyDelimiter, + keyNullLiteral, + serializationSchema, + contentType, + failureHandler, + sinkOptions); + } + + @Override + protected ElasticsearchSink.Builder> createBuilder( + ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { + builder = super.createBuilder(upsertSinkFunction, httpHosts); + return builder; + } + } + + private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { + + @Override + public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { + throw new UnsupportedOperationException(); + } + } + + private static class DataStreamMock extends DataStream> { + + public SinkFunction sinkFunction; + + public DataStreamMock( + StreamExecutionEnvironment environment, + TypeInformation> outType) { + super(environment, new TransformationMock("name", outType, 1)); + } + + @Override + public DataStreamSink> addSink( + SinkFunction> sinkFunction) { + this.sinkFunction = sinkFunction; + return super.addSink(sinkFunction); + } + } + + private static class TransformationMock extends Transformation> { + + public TransformationMock( + String name, TypeInformation> outputType, int parallelism) { + super(name, outputType, parallelism); + } + + @Override + public List> getTransitivePredecessors() { + return null; + } + + @Override + public List> getInputs() { + return Collections.emptyList(); + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 3ba3a8b8..b651e6a2 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -37,94 +37,100 @@ import java.util.ArrayList; import java.util.List; -/** - * IT cases for the {@link ElasticsearchSink}. - */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName - .parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("7.5.1")); - - @Override - protected String getClusterName() { - return "docker-cluster"; - } - - @Override - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = getClusterName(); - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings) - .addTransportAddress(transportAddress); - } - - @Test - public void testElasticsearchSink() throws Exception { - runElasticsearchSinkTest(); - } - - @Test - public void testElasticsearchSinkWithSmile() throws Exception { - runElasticsearchSinkSmileTest(); - } - - @Test - public void testNullAddresses() { - runNullAddressesTest(); - } - - @Test - public void testEmptyAddresses() { - runEmptyAddressesTest(); - } - - @Test - public void testInvalidElasticsearchCluster() throws Exception{ - runInvalidElasticsearchClusterTest(); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - int bulkFlushMaxActions, - String clusterName, - List httpHosts, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { - - ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkFlushMaxActions(bulkFlushMaxActions); - - return builder.build(); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction) { - - return createElasticsearchSinkForNode( - bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, elasticsearchContainer.getHttpHostAddress()); - } - - @Override - protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( - int bulkFlushMaxActions, - String clusterName, - ElasticsearchSinkFunction> elasticsearchSinkFunction, - String hostAddress) { - - ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(HttpHost.create(hostAddress)); - - ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkFlushMaxActions(bulkFlushMaxActions); - - return builder.build(); - } +/** IT cases for the {@link ElasticsearchSink}. */ +public class ElasticsearchSinkITCase + extends ElasticsearchSinkTestBase { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer( + DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") + .withTag("7.5.1")); + + @Override + protected String getClusterName() { + return "docker-cluster"; + } + + @Override + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = getClusterName(); + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); + } + + @Test + public void testElasticsearchSinkWithSmile() throws Exception { + runElasticsearchSinkSmileTest(); + } + + @Test + public void testNullAddresses() { + runNullAddressesTest(); + } + + @Test + public void testEmptyAddresses() { + runEmptyAddressesTest(); + } + + @Test + public void testInvalidElasticsearchCluster() throws Exception { + runInvalidElasticsearchClusterTest(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSink( + int bulkFlushMaxActions, + String clusterName, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForEmbeddedNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return createElasticsearchSinkForNode( + bulkFlushMaxActions, + clusterName, + elasticsearchSinkFunction, + elasticsearchContainer.getHttpHostAddress()); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> + createElasticsearchSinkForNode( + int bulkFlushMaxActions, + String clusterName, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String hostAddress) { + + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(HttpHost.create(hostAddress)); + + ElasticsearchSink.Builder> builder = + new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); + + return builder.build(); + } } From 5143fe4889afaddb224199ec69d51ab775792188 Mon Sep 17 00:00:00 2001 From: Jie Wang Date: Tue, 22 Dec 2020 22:42:30 +0800 Subject: [PATCH 123/207] [FLINK-20711][docs] Fix failure-handler parameter enumeration value typo in ES connector doc This closes #14467 --- .../connectors/elasticsearch/table/ElasticsearchOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index 7f437f95..c40b94cd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -89,7 +89,7 @@ public enum BackOffType { text( "\"ignore\" (ignores failures and drops the request),"), text( - "\"retry_rejected\" (re-adds requests that have failed due to queue capacity saturation),"), + "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation),"), text( "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) .build()); From 947c273f4db217b2b56d711008ff817c2737a3b3 Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 9 Jan 2021 20:24:25 +0800 Subject: [PATCH 124/207] [FLINK-20906][legal] Update copyright year to 2021 for NOTICE files. This closes #14598 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index b77afb5a..b8f64c22 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch6 -Copyright 2014-2020 The Apache Software Foundation +Copyright 2014-2021 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 5a8708ec..f6ebf8b7 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -1,5 +1,5 @@ flink-sql-connector-elasticsearch7 -Copyright 2014-2020 The Apache Software Foundation +Copyright 2014-2021 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From efbd46315a26f9b5ad51d46d832ac771913f8e21 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 12 Jan 2021 10:13:24 +0100 Subject: [PATCH 125/207] [FLINK-20862] Make raw types of TypeInformation resolvable This closes #14619. --- .../table/Elasticsearch6DynamicSinkFactoryTest.java | 7 +++++-- .../table/Elasticsearch7DynamicSinkFactoryTest.java | 7 +++++-- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 0d1f7510..d638dd81 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; @@ -190,7 +190,10 @@ public void validatePrimaryKeyOnIllegalColumn() { DataTypes.FIELD( "a", DataTypes.BIGINT())) .notNull()) - .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field( + "f", + DataTypes.RAW(Void.class, VoidSerializer.INSTANCE) + .notNull()) .field("g", DataTypes.BYTES().notNull()) .primaryKey("a", "b", "c", "d", "e", "f", "g") .build()) diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 14aef72f..e81b5ee4 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; @@ -183,7 +183,10 @@ public void validatePrimaryKeyOnIllegalColumn() { DataTypes.FIELD( "a", DataTypes.BIGINT())) .notNull()) - .field("f", DataTypes.RAW(Types.BIG_INT).notNull()) + .field( + "f", + DataTypes.RAW(Void.class, VoidSerializer.INSTANCE) + .notNull()) .field("g", DataTypes.BYTES().notNull()) .primaryKey("a", "b", "c", "d", "e", "f", "g") .build()) From f10f96730a333a4493ee83a995b920206d2b4135 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 18 Jan 2021 11:35:56 +0100 Subject: [PATCH 126/207] [FLINK-21009] Can not disable certain options in Elasticsearch 7 connector --- .../elasticsearch/ElasticsearchSinkBase.java | 27 ++++++++++++++++--- .../elasticsearch6/ElasticsearchSink.java | 20 +++++++++++--- .../elasticsearch7/ElasticsearchSink.java | 16 ++++++----- 3 files changed, 50 insertions(+), 13 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 15c64624..625adca5 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -386,13 +386,11 @@ protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { } if (bulkProcessorFlushMaxSizeMb != null) { - bulkProcessorBuilder.setBulkSize( - new ByteSizeValue(bulkProcessorFlushMaxSizeMb, ByteSizeUnit.MB)); + configureBulkSize(bulkProcessorBuilder); } if (bulkProcessorFlushIntervalMillis != null) { - bulkProcessorBuilder.setFlushInterval( - TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + configureFlushInterval(bulkProcessorBuilder); } // if backoff retrying is disabled, bulkProcessorFlushBackoffPolicy will be null @@ -402,6 +400,27 @@ protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { return bulkProcessorBuilder.build(); } + private void configureBulkSize(BulkProcessor.Builder bulkProcessorBuilder) { + final ByteSizeUnit sizeUnit; + if (bulkProcessorFlushMaxSizeMb == -1) { + // bulk size can be disabled with -1, however the ByteSizeValue constructor accepts -1 + // only with BYTES as the size unit + sizeUnit = ByteSizeUnit.BYTES; + } else { + sizeUnit = ByteSizeUnit.MB; + } + bulkProcessorBuilder.setBulkSize(new ByteSizeValue(bulkProcessorFlushMaxSizeMb, sizeUnit)); + } + + private void configureFlushInterval(BulkProcessor.Builder bulkProcessorBuilder) { + if (bulkProcessorFlushIntervalMillis == -1) { + bulkProcessorBuilder.setFlushInterval(null); + } else { + bulkProcessorBuilder.setFlushInterval( + TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + } + } + private void checkErrorAndRethrow() { Throwable cause = failureThrowable.get(); if (cause != null) { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index b899be06..86fb7744 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -109,31 +109,45 @@ public Builder( } /** - * Sets the maximum number of actions to buffer for each bulk request. + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. * * @param numMaxActions the maximum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkRequestsConfig.put( CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); } /** - * Sets the maximum size of buffered actions, in mb, per bulk request. + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. * * @param maxSizeMb the maximum size of buffered actions, in mb. */ public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkRequestsConfig.put( CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); } /** - * Sets the bulk flush interval, in milliseconds. + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. * * @param intervalMillis the bulk flush interval, in milliseconds. */ public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + this.bulkRequestsConfig.put( CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java index 3ac23435..92b17d04 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -109,39 +109,43 @@ public Builder( } /** - * Sets the maximum number of actions to buffer for each bulk request. + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. * * @param numMaxActions the maximum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { Preconditions.checkArgument( - numMaxActions > 0, "Max number of buffered actions must be larger than 0."); + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); this.bulkRequestsConfig.put( CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); } /** - * Sets the maximum size of buffered actions, in mb, per bulk request. + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. * * @param maxSizeMb the maximum size of buffered actions, in mb. */ public void setBulkFlushMaxSizeMb(int maxSizeMb) { Preconditions.checkArgument( - maxSizeMb > 0, "Max size of buffered actions must be larger than 0."); + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); this.bulkRequestsConfig.put( CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); } /** - * Sets the bulk flush interval, in milliseconds. + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. * * @param intervalMillis the bulk flush interval, in milliseconds. */ public void setBulkFlushInterval(long intervalMillis) { Preconditions.checkArgument( - intervalMillis >= 0, + intervalMillis == -1 || intervalMillis >= 0, "Interval (in milliseconds) between each flush must be larger than or equal to 0."); this.bulkRequestsConfig.put( From 7c4af0ed76c932b21c0286f54d007dba89a62881 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Thu, 21 Jan 2021 19:50:46 +0800 Subject: [PATCH 127/207] [FLINK-21020][build] Bump Jackson to 2.12.1 --- .../src/main/resources/META-INF/NOTICE | 10 ++++++---- .../src/main/resources/META-INF/NOTICE | 10 ++++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index b8f64c22..ff2a4bbd 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -6,10 +6,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 +- com.fasterxml.jackson.core:jackson-core:2.12.1 +- com.fasterxml.jackson.core:jackson-databind:2.12.1 +- com.fasterxml.jackson.core:jackson-annotations:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 - commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index f6ebf8b7..e3e63b1e 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -7,10 +7,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.carrotsearch:hppc:0.8.1 -- com.fasterxml.jackson.core:jackson-core:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.10.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.10.1 +- com.fasterxml.jackson.core:jackson-core:2.12.1 +- com.fasterxml.jackson.core:jackson-databind:2.12.1 +- com.fasterxml.jackson.core:jackson-annotations:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 - com.github.spullara.mustache.java:compiler:0.9.6 - commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 From c7e11ea2f4c0fb750bcc4bbbc900f7c1769df4e7 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 4 Feb 2021 09:24:07 +0100 Subject: [PATCH 128/207] [FLINK-21277] Bump testcontainers version --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 875a3c9d..85bf6796 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -86,7 +86,7 @@ under the License. org.testcontainers elasticsearch - 1.15.0 + 1.15.1 test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index bf213f62..f12f68fb 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -86,7 +86,7 @@ under the License. org.testcontainers elasticsearch - 1.15.0 + 1.15.1 test From d723c3e5c26ce44d8c9cbcf16b4bd9c2fcaab6cc Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 15 Mar 2021 09:38:15 +0100 Subject: [PATCH 129/207] [FLINK-21396][table-planner-blink] Use ResolvedCatalogTable within the planner Updates the planner to only use instances of ResolvedCatalogTable with ResolvedSchema. This closes #15217. --- .../ElasticsearchUpsertTableSinkFactoryBase.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java index dec4b30f..db574bb2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java @@ -53,6 +53,7 @@ import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; +import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; @@ -161,6 +162,9 @@ public List supportedProperties() { properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); + // comment + properties.add(COMMENT); + // format wildcard properties.add(FORMAT + ".*"); From 6cdd3f28a913259b525f96e1b5d64e243187c52c Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Thu, 18 Mar 2021 12:13:17 +0100 Subject: [PATCH 130/207] [FLINK-21913][table][connectors] Update DynamicTableFactory.Context to use ResolvedCatalogTable This closes #15316. --- .../elasticsearch/table/TestContext.java | 59 ++++++------ .../Elasticsearch6DynamicSinkFactoryTest.java | 87 +++++++++--------- .../Elasticsearch6DynamicSinkITCase.java | 29 +++--- .../Elasticsearch7DynamicSinkFactoryTest.java | 92 ++++++++++--------- .../Elasticsearch7DynamicSinkITCase.java | 30 +++--- 5 files changed, 154 insertions(+), 143 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java index 5b72ce21..a6dba08a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -18,61 +18,54 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import java.util.Collections; import java.util.HashMap; import java.util.Map; /** A utility class for mocking {@link DynamicTableFactory.Context}. */ class TestContext { - private TableSchema schema; - private Map properties = new HashMap<>(); + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); public static TestContext context() { return new TestContext(); } - public TestContext withSchema(TableSchema schema) { + public TestContext withSchema(ResolvedSchema schema) { this.schema = schema; return this; } DynamicTableFactory.Context build() { - return new DynamicTableFactory.Context() { - @Override - public ObjectIdentifier getObjectIdentifier() { - return null; - } - - @Override - public CatalogTable getCatalogTable() { - return new CatalogTableImpl(schema, properties, ""); - } - - @Override - public ReadableConfig getConfiguration() { - return null; - } - - @Override - public ClassLoader getClassLoader() { - return TestContext.class.getClassLoader(); - } - - @Override - public boolean isTemporary() { - return false; - } - }; + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + new Configuration(), + TestContext.class.getClassLoader(), + false); } public TestContext withOption(String key, String value) { - properties.put(key, value); + options.put(key, value); return this; } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index d638dd81..d3558108 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -20,13 +20,18 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Arrays; +import java.util.Collections; + import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ @@ -46,10 +51,7 @@ public void validateEmptyConfiguration() { + "document-type\n" + "hosts\n" + "index"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) - .build()); + sinkFactory.createDynamicTableSink(context().build()); } @Test @@ -60,7 +62,6 @@ public void validateWrongIndex() { thrown.expectMessage("'index' must not be empty"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption("index", "") .withOption("document-type", "MyType") .withOption("hosts", "http://localhost:12345") @@ -76,7 +77,6 @@ public void validateWrongHosts() { "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption("index", "MyIndex") .withOption("document-type", "MyType") .withOption("hosts", "wrong-host") @@ -92,7 +92,6 @@ public void validateWrongFlushSize() { "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( @@ -109,7 +108,6 @@ public void validateWrongRetries() { thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( @@ -128,7 +126,6 @@ public void validateWrongMaxActions() { thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( @@ -145,7 +142,6 @@ public void validateWrongBackoffDelay() { thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( @@ -168,35 +164,45 @@ public void validatePrimaryKeyOnIllegalColumn() { sinkFactory.createDynamicTableSink( context() .withSchema( - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field( - "b", - DataTypes.ARRAY(DataTypes.BIGINT().notNull()) - .notNull()) - .field( - "c", - DataTypes.MAP( - DataTypes.BIGINT(), - DataTypes.STRING()) - .notNull()) - .field( - "d", - DataTypes.MULTISET(DataTypes.BIGINT().notNull()) - .notNull()) - .field( - "e", - DataTypes.ROW( - DataTypes.FIELD( - "a", DataTypes.BIGINT())) - .notNull()) - .field( - "f", - DataTypes.RAW(Void.class, VoidSerializer.INSTANCE) - .notNull()) - .field("g", DataTypes.BYTES().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g") - .build()) + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -214,7 +220,6 @@ public void validateWrongCredential() { "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index cb8457b9..e43b0c53 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -25,7 +25,9 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.GenericRowData; @@ -50,6 +52,8 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -79,17 +83,18 @@ protected final Client getClient() { @Test public void testWritingDocuments() throws Exception { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.TIME()) - .field("c", DataTypes.STRING().notNull()) - .field("d", DataTypes.FLOAT()) - .field("e", DataTypes.TINYINT().notNull()) - .field("f", DataTypes.DATE()) - .field("g", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "g") - .build(); + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); GenericRowData rowData = GenericRowData.of( 1L, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index e81b5ee4..7480254c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -20,13 +20,18 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.Arrays; +import java.util.Collections; + import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ @@ -45,10 +50,7 @@ public void validateEmptyConfiguration() { + "\n" + "hosts\n" + "index"); - sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) - .build()); + sinkFactory.createDynamicTableSink(context().build()); } @Test @@ -59,7 +61,6 @@ public void validateWrongIndex() { thrown.expectMessage("'index' must not be empty"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption("index", "") .withOption("hosts", "http://localhost:12345") .build()); @@ -73,11 +74,7 @@ public void validateWrongHosts() { thrown.expectMessage( "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); sinkFactory.createDynamicTableSink( - context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) - .withOption("index", "MyIndex") - .withOption("hosts", "wrong-host") - .build()); + context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); } @Test @@ -89,7 +86,6 @@ public void validateWrongFlushSize() { "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -105,7 +101,6 @@ public void validateWrongRetries() { thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -123,7 +118,6 @@ public void validateWrongMaxActions() { thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -139,7 +133,6 @@ public void validateWrongBackoffDelay() { thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -161,35 +154,45 @@ public void validatePrimaryKeyOnIllegalColumn() { sinkFactory.createDynamicTableSink( context() .withSchema( - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field( - "b", - DataTypes.ARRAY(DataTypes.BIGINT().notNull()) - .notNull()) - .field( - "c", - DataTypes.MAP( - DataTypes.BIGINT(), - DataTypes.STRING()) - .notNull()) - .field( - "d", - DataTypes.MULTISET(DataTypes.BIGINT().notNull()) - .notNull()) - .field( - "e", - DataTypes.ROW( - DataTypes.FIELD( - "a", DataTypes.BIGINT())) - .notNull()) - .field( - "f", - DataTypes.RAW(Void.class, VoidSerializer.INSTANCE) - .notNull()) - .field("g", DataTypes.BYTES().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g") - .build()) + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") @@ -207,7 +210,6 @@ public void validateWrongCredential() { "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); sinkFactory.createDynamicTableSink( context() - .withSchema(TableSchema.builder().field("a", DataTypes.TIME()).build()) .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") .withOption( ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index b92f34e8..e7a9271c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -25,7 +25,9 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkFunctionProvider; import org.apache.flink.table.data.GenericRowData; @@ -50,6 +52,8 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -79,17 +83,19 @@ protected final Client getClient() { @Test public void testWritingDocuments() throws Exception { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.TIME()) - .field("c", DataTypes.STRING().notNull()) - .field("d", DataTypes.FLOAT()) - .field("e", DataTypes.TINYINT().notNull()) - .field("f", DataTypes.DATE()) - .field("g", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "g") - .build(); + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = GenericRowData.of( 1L, From 826b67c416c91eb64f7a064ac9315c6048e83fb8 Mon Sep 17 00:00:00 2001 From: GuoWei Ma Date: Sun, 18 Apr 2021 20:19:52 +0800 Subject: [PATCH 131/207] Update version to 1.14-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index efed549b..0c069fbe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 85bf6796..afdb3fec 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index f12f68fb..ea9ed8ec 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 80598c58..e83cfe2e 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 2c0d3d9c..255af93f 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.13-SNAPSHOT + 1.14-SNAPSHOT .. From 8ed425e8f19f0c1ed9dd6f29c331c7d1d454fb77 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Mon, 26 Apr 2021 09:11:12 +0200 Subject: [PATCH 132/207] [FLINK-22471][connector-elasticsearch] Remove commads from list This is a responsibility of the Formatter implementation. --- .../elasticsearch/table/ElasticsearchOptions.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index c40b94cd..258443c0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -85,11 +85,11 @@ public enum BackOffType { "Failure handling strategy in case a request to Elasticsearch fails") .list( text( - "\"fail\" (throws an exception if a request fails and thus causes a job failure),"), + "\"fail\" (throws an exception if a request fails and thus causes a job failure)"), text( - "\"ignore\" (ignores failures and drops the request),"), + "\"ignore\" (ignores failures and drops the request)"), text( - "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation),"), + "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation)"), text( "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) .build()); From 320bb0703c4215e7080ffeba43edbfab1da8ec25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Mon, 26 Apr 2021 09:11:29 +0200 Subject: [PATCH 133/207] [FLINK-22471][connector-elasticsearch] Do not repeat default value --- .../connectors/elasticsearch/table/ElasticsearchOptions.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java index 258443c0..a595055a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java @@ -143,9 +143,8 @@ public enum BackOffType { .stringType() .defaultValue("json") .withDescription( - "Elasticsearch connector requires to specify a format.\n" - + "The format must produce a valid json document. \n" - + "By default uses built-in 'json' format. Please refer to Table Formats section for more details."); + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); private ElasticsearchOptions() {} } From 26c2ac20db0dfb4ecc145b5aab017e99f1e666f4 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 31 May 2021 14:52:49 +0200 Subject: [PATCH 134/207] [FLINK-22810][connector-elasticsearch] Drop usages of legacy planner in Elasticsearch modules This closes #16031. --- .../pom.xml | 44 +++++-------- .../ElasticsearchUpsertTableSinkBase.java | 3 +- ...csearchUpsertTableSinkFactoryTestBase.java | 64 ++++++++++--------- .../flink-connector-elasticsearch6/pom.xml | 38 +++++------ .../flink-connector-elasticsearch7/pom.xml | 30 ++++----- 5 files changed, 79 insertions(+), 100 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 0c069fbe..ea25c68e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -42,7 +42,7 @@ under the License. - + org.apache.flink @@ -51,6 +51,19 @@ under the License. provided + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.elasticsearch elasticsearch @@ -70,25 +83,7 @@ under the License. - - - - org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} - ${project.version} - provided - true - - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - provided - true - - - + org.apache.flink @@ -113,15 +108,6 @@ under the License. test-jar - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - - org.apache.flink diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java index 7037bc7f..5d47dacb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java @@ -33,7 +33,6 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.sinks.TableSink; import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.table.typeutils.TypeCheckUtils; import org.apache.flink.table.utils.TableConnectorUtils; import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.types.Row; @@ -305,7 +304,7 @@ private void validateKeyTypes(int[] keyFieldIndices) { final TypeInformation[] types = getFieldTypes(); for (int keyFieldIndex : keyFieldIndices) { final TypeInformation type = types[keyFieldIndex]; - if (!TypeCheckUtils.isSimpleStringRepresentation(type)) { + if (!type.isKeyType()) { throw new ValidationException( "Only simple types that can be safely converted into a string representation " + "can be used as keys. But was: " diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java index 03e5b241..a67953d8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java @@ -26,10 +26,6 @@ import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.descriptors.Elasticsearch; -import org.apache.flink.table.descriptors.Json; -import org.apache.flink.table.descriptors.Schema; -import org.apache.flink.table.descriptors.TestTableDescriptor; import org.apache.flink.table.factories.StreamTableSinkFactory; import org.apache.flink.table.factories.TableFactoryService; import org.apache.flink.table.sinks.TableSink; @@ -160,32 +156,40 @@ protected Map createTestSinkOptions() { } protected Map createElasticSearchProperties() { - return new TestTableDescriptor( - new Elasticsearch() - .version(getElasticsearchVersion()) - .host(HOSTNAME, PORT, SCHEMA) - .index(INDEX) - .documentType(DOC_TYPE) - .keyDelimiter(KEY_DELIMITER) - .keyNullLiteral(KEY_NULL_LITERAL) - .bulkFlushBackoffExponential() - .bulkFlushBackoffDelay(123L) - .bulkFlushBackoffMaxRetries(3) - .bulkFlushInterval(100L) - .bulkFlushMaxActions(1000) - .bulkFlushMaxSize("1 MB") - .failureHandlerCustom(DummyFailureHandler.class) - .connectionMaxRetryTimeout(100) - .connectionPathPrefix("/myapp")) - .withFormat(new Json().deriveSchema()) - .withSchema( - new Schema() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3))) - .inUpsertMode() - .toProperties(); + final Map map = new HashMap<>(); + map.put("connector.bulk-flush.backoff.type", "exponential"); + map.put("connector.bulk-flush.max-size", "1 mb"); + map.put("schema.0.data-type", "BIGINT"); + map.put("schema.1.name", "fruit_name"); + map.put("connector.property-version", "1"); + map.put("connector.bulk-flush.backoff.max-retries", "3"); + map.put("schema.3.data-type", "TIMESTAMP(3)"); + map.put("connector.document-type", "MyType"); + map.put("schema.3.name", "ts"); + map.put("connector.index", "MyIndex"); + map.put("schema.0.name", "key"); + map.put("connector.bulk-flush.backoff.delay", "123"); + map.put("connector.bulk-flush.max-actions", "1000"); + map.put("schema.2.name", "count"); + map.put("update-mode", "upsert"); + map.put( + "connector.failure-handler-class", + ElasticsearchUpsertTableSinkFactoryTestBase.DummyFailureHandler.class.getName()); + map.put("format.type", "json"); + map.put("schema.1.data-type", "VARCHAR(2147483647)"); + map.put("connector.version", getElasticsearchVersion()); + map.put("connector.bulk-flush.interval", "100"); + map.put("schema.2.data-type", "DECIMAL(10, 4)"); + map.put("connector.hosts", "https://host1:1234"); + map.put("connector.failure-handler", "custom"); + map.put("format.property-version", "1"); + map.put("format.derive-schema", "true"); + map.put("connector.type", "elasticsearch"); + map.put("connector.key-null-literal", ""); + map.put("connector.key-delimiter", "#"); + map.put("connector.connection-path-prefix", "/myapp"); + map.put("connector.connection-max-retry-timeout", "100"); + return map; } // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index afdb3fec..b717dccc 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -42,7 +42,7 @@ under the License. - + org.apache.flink @@ -51,6 +51,19 @@ under the License. provided + + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${project.version} + provided + true + + + + org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} @@ -64,6 +77,8 @@ under the License. + + org.elasticsearch.client @@ -71,17 +86,7 @@ under the License. ${elasticsearch.version} - - - - org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} - ${project.version} - provided - true - - - + org.testcontainers @@ -143,15 +148,6 @@ under the License. provided - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - - org.apache.flink diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index ea9ed8ec..63d6b629 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -42,7 +42,7 @@ under the License. - + org.apache.flink @@ -51,6 +51,8 @@ under the License. provided + + org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} @@ -64,14 +66,8 @@ under the License. - - - org.elasticsearch.client - elasticsearch-rest-high-level-client - ${elasticsearch.version} - - + org.apache.flink @@ -81,7 +77,14 @@ under the License. true - + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + org.testcontainers @@ -137,15 +140,6 @@ under the License. test - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${project.version} - test-jar - test - - org.apache.flink From a4eaf30744ab94313d7a30871ac954c78af6a7f9 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 7 Jun 2021 17:37:43 +0200 Subject: [PATCH 135/207] [FLINK-22744][table] Update and simplify EnvironmentSettings --- .../table/Elasticsearch6DynamicSinkITCase.java | 18 +++--------------- .../table/Elasticsearch7DynamicSinkITCase.java | 18 +++--------------- 2 files changed, 6 insertions(+), 30 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index e43b0c53..c3ed40d9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -160,11 +160,7 @@ public void testWritingDocuments() throws Exception { @Test public void testWritingDocumentsFromTableApi() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "table-api"; String myType = "MyType"; @@ -228,11 +224,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { @Test public void testWritingDocumentsNoPrimaryKey() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "no-primary-key"; String myType = "MyType"; @@ -328,11 +320,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { @Test public void testWritingDocumentsWithDynamicIndex() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "dynamic-index-{b|yyyy-MM-dd}"; String myType = "MyType"; diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index e7a9271c..5efaba96 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -154,11 +154,7 @@ public void testWritingDocuments() throws Exception { @Test public void testWritingDocumentsFromTableApi() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "table-api"; tableEnvironment.executeSql( @@ -216,11 +212,7 @@ public void testWritingDocumentsFromTableApi() throws Exception { @Test public void testWritingDocumentsNoPrimaryKey() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "no-primary-key"; tableEnvironment.executeSql( @@ -312,11 +304,7 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { @Test public void testWritingDocumentsWithDynamicIndex() throws Exception { TableEnvironment tableEnvironment = - TableEnvironment.create( - EnvironmentSettings.newInstance() - .useBlinkPlanner() - .inStreamingMode() - .build()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); String index = "dynamic-index-{b|yyyy-MM-dd}"; tableEnvironment.executeSql( From 04be9bf3668ab7011ec1cf0b951b503f0d68de84 Mon Sep 17 00:00:00 2001 From: Yangyang ZHANG Date: Thu, 24 Jun 2021 15:53:38 +0800 Subject: [PATCH 136/207] [FLINK-21448] Add dependency for ChangelogStateBackend ITTests --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 7 +++++++ flink-connectors/flink-connector-elasticsearch7/pom.xml | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index b717dccc..c8050ca6 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -110,6 +110,13 @@ under the License. test-jar + + org.apache.flink + flink-statebackend-changelog_${scala.binary.version} + ${project.version} + test + + org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 63d6b629..40a1c140 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -108,6 +108,13 @@ under the License. test-jar + + org.apache.flink + flink-statebackend-changelog_${scala.binary.version} + ${project.version} + test + + org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} From 2b1b66d7c8708aff6b611a0d67f922c0405ef9dc Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Tue, 6 Jul 2021 14:22:24 +0200 Subject: [PATCH 137/207] [FLINK-22879][table] Rename flink-table-planner-blink to flink-table-planner It might be required to update job dependencies. Note that flink-table-planner used to contain the legacy planner before Flink 1.14 and now cotains the only officially supported planner (i.e. previously known as 'Blink' planner). This closes #16386. --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index c8050ca6..b8c89dde 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -158,7 +158,7 @@ under the License. org.apache.flink - flink-table-planner-blink_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 40a1c140..ecac13d1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -150,7 +150,7 @@ under the License. org.apache.flink - flink-table-planner-blink_${scala.binary.version} + flink-table-planner_${scala.binary.version} ${project.version} test From bfa43ac9c4a024bbe9573c7604291c9a34f7364e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Wed, 30 Jun 2021 10:32:46 +0200 Subject: [PATCH 138/207] [FLINK-23064][connector-elasticsearch] Make connector options PublicEvolving --- .../table/ElasticsearchConfiguration.java | 31 ++++----- ...ava => ElasticsearchConnectorOptions.java} | 49 ++++++++++---- .../table/Elasticsearch6Configuration.java | 2 +- .../Elasticsearch6DynamicSinkFactory.java | 34 +++++----- .../Elasticsearch6DynamicSinkFactoryTest.java | 66 ++++++++++++------- .../Elasticsearch6DynamicSinkITCase.java | 41 +++++++----- .../table/Elasticsearch6DynamicSinkTest.java | 49 ++++++++------ .../table/Elasticsearch7Configuration.java | 2 +- .../Elasticsearch7DynamicSinkFactory.java | 32 ++++----- .../Elasticsearch7DynamicSinkFactoryTest.java | 51 ++++++++------ .../Elasticsearch7DynamicSinkITCase.java | 32 +++++---- .../table/Elasticsearch7DynamicSinkTest.java | 49 ++++++++------ 12 files changed, 263 insertions(+), 175 deletions(-) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/{ElasticsearchOptions.java => ElasticsearchConnectorOptions.java} (94%) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index e1ffab77..04c76333 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -32,13 +32,13 @@ import java.util.Objects; import java.util.Optional; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; /** Accessor methods to elasticsearch options. */ @Internal @@ -80,17 +80,18 @@ public ActionRequestFailureHandler getFailureHandler() { } public String getDocumentType() { - return config.get(ElasticsearchOptions.DOCUMENT_TYPE_OPTION); + return config.get(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION); } public int getBulkFlushMaxActions() { - int maxActions = config.get(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + int maxActions = config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. return maxActions == 0 ? -1 : maxActions; } public long getBulkFlushMaxByteSize() { - long maxSize = config.get(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + long maxSize = + config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. return maxSize == 0 ? -1 : maxSize; } @@ -111,7 +112,7 @@ public Optional getPassword() { public boolean isBulkFlushBackoffEnabled() { return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) - != ElasticsearchOptions.BackOffType.DISABLED; + != ElasticsearchConnectorOptions.BackOffType.DISABLED; } public Optional getBulkFlushBackoffType() { @@ -134,19 +135,19 @@ public Optional getBulkFlushBackoffDelay() { } public boolean isDisableFlushOnCheckpoint() { - return !config.get(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION); + return !config.get(ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION); } public String getIndex() { - return config.get(ElasticsearchOptions.INDEX_OPTION); + return config.get(ElasticsearchConnectorOptions.INDEX_OPTION); } public String getKeyDelimiter() { - return config.get(ElasticsearchOptions.KEY_DELIMITER_OPTION); + return config.get(ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION); } public Optional getPathPrefix() { - return config.getOptional(ElasticsearchOptions.CONNECTION_PATH_PREFIX); + return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java similarity index 94% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index a595055a..1b0fc5e4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.MemorySize; @@ -29,19 +30,9 @@ import static org.apache.flink.configuration.description.TextElement.text; -/** - * Options for {@link org.apache.flink.table.factories.DynamicTableSinkFactory} for Elasticsearch. - */ -public class ElasticsearchOptions { - /** - * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code - * DISABLED} option. - */ - public enum BackOffType { - DISABLED, - CONSTANT, - EXPONENTIAL - } +/** Options for the Elasticsearch connector. */ +@PublicEvolving +public class ElasticsearchConnectorOptions { public static final ConfigOption> HOSTS_OPTION = ConfigOptions.key("hosts") @@ -49,32 +40,38 @@ public enum BackOffType { .asList() .noDefaultValue() .withDescription("Elasticsearch hosts to connect to."); + public static final ConfigOption INDEX_OPTION = ConfigOptions.key("index") .stringType() .noDefaultValue() .withDescription("Elasticsearch index for every record."); + public static final ConfigOption DOCUMENT_TYPE_OPTION = ConfigOptions.key("document-type") .stringType() .noDefaultValue() .withDescription("Elasticsearch document type."); + public static final ConfigOption PASSWORD_OPTION = ConfigOptions.key("password") .stringType() .noDefaultValue() .withDescription("Password used to connect to Elasticsearch instance."); + public static final ConfigOption USERNAME_OPTION = ConfigOptions.key("username") .stringType() .noDefaultValue() .withDescription("Username used to connect to Elasticsearch instance."); + public static final ConfigOption KEY_DELIMITER_OPTION = ConfigOptions.key("document-id.key-delimiter") .stringType() .defaultValue("_") .withDescription( "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + public static final ConfigOption FAILURE_HANDLER_OPTION = ConfigOptions.key("failure-handler") .stringType() @@ -93,51 +90,61 @@ public enum BackOffType { text( "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) .build()); + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = ConfigOptions.key("sink.flush-on-checkpoint") .booleanType() .defaultValue(true) .withDescription("Disables flushing on checkpoint"); + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = ConfigOptions.key("sink.bulk-flush.max-actions") .intType() .defaultValue(1000) .withDescription("Maximum number of actions to buffer for each bulk request."); + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = ConfigOptions.key("sink.bulk-flush.max-size") .memoryType() .defaultValue(MemorySize.parse("2mb")) .withDescription("Maximum size of buffered actions per bulk request"); + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = ConfigOptions.key("sink.bulk-flush.interval") .durationType() .defaultValue(Duration.ofSeconds(1)) .withDescription("Bulk flush interval"); + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = ConfigOptions.key("sink.bulk-flush.backoff.strategy") .enumType(BackOffType.class) .defaultValue(BackOffType.DISABLED) .withDescription("Backoff strategy"); + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = ConfigOptions.key("sink.bulk-flush.backoff.max-retries") .intType() .noDefaultValue() .withDescription("Maximum number of retries."); + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = ConfigOptions.key("sink.bulk-flush.backoff.delay") .durationType() .noDefaultValue() .withDescription("Delay between each backoff attempt."); + public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = ConfigOptions.key("connection.max-retry-timeout") .durationType() .noDefaultValue() .withDescription("Maximum timeout between retries."); + public static final ConfigOption CONNECTION_PATH_PREFIX = ConfigOptions.key("connection.path-prefix") .stringType() .noDefaultValue() .withDescription("Prefix string to be added to every REST communication."); + public static final ConfigOption FORMAT_OPTION = ConfigOptions.key("format") .stringType() @@ -146,5 +153,19 @@ public enum BackOffType { "The format must produce a valid JSON document. " + "Please refer to the documentation on formats for more details."); - private ElasticsearchOptions() {} + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code + * DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } + + private ElasticsearchConnectorOptions() {} } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java index af03eedc..8b833216 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -27,7 +27,7 @@ import java.util.List; import java.util.stream.Collectors; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; /** Elasticsearch 6 specific configuration. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 6fc4030c..32d0ccd4 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -38,23 +38,23 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.DOCUMENT_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; /** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index d3558108..25bcbbfb 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -92,11 +92,15 @@ public void validateWrongFlushSize() { "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") .build()); } @@ -108,12 +112,15 @@ public void validateWrongRetries() { thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), "0") .build()); } @@ -126,11 +133,15 @@ public void validateWrongMaxActions() { thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") .build()); } @@ -142,12 +153,15 @@ public void validateWrongBackoffDelay() { thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") .build()); } @@ -203,11 +217,13 @@ public void validatePrimaryKeyOnIllegalColumn() { UniqueConstraint.primaryKey( "name", Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") .build()); } @@ -220,12 +236,14 @@ public void validateWrongCredential() { "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") .build()); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index c3ed40d9..e9795015 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -117,17 +117,20 @@ public void testWritingDocuments() throws Exception { context() .withSchema(schema) .withOption( - ElasticsearchOptions.INDEX_OPTION.key(), + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), index) .withOption( - ElasticsearchOptions.DOCUMENT_TYPE_OPTION + ElasticsearchConnectorOptions + .DOCUMENT_TYPE_OPTION .key(), myType) .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), elasticsearchContainer.getHttpHostAddress()) .withOption( - ElasticsearchOptions + ElasticsearchConnectorOptions .FLUSH_ON_CHECKPOINT_OPTION .key(), "false") @@ -179,17 +182,19 @@ public void testWritingDocumentsFromTableApi() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment @@ -241,17 +246,19 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment @@ -333,17 +340,19 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), myType) + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index 5d585933..f654222e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -95,10 +95,11 @@ public void testBuilder() { public void testDefaultConfig() { final TableSchema schema = createTestSchema(); Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); BuilderProvider provider = new BuilderProvider(); final Elasticsearch6DynamicSink testSink = @@ -125,12 +126,13 @@ public void testDefaultConfig() { public void testAuthConfig() { final TableSchema schema = createTestSchema(); Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); BuilderProvider provider = new BuilderProvider(); final Elasticsearch6DynamicSink testSink = @@ -157,23 +159,30 @@ public void testAuthConfig() { private Configuration getConfig() { Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); configuration.setString( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); configuration.setString( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); configuration.setString( - ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); - configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); return configuration; } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java index 733f774a..6bd28cf4 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -27,7 +27,7 @@ import java.util.List; import java.util.stream.Collectors; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; /** Elasticsearch 7 specific configuration. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 96344194..e5fed88c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -38,22 +38,22 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.CONNECTION_PATH_PREFIX; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FAILURE_HANDLER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchOptions.USERNAME_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; /** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 7480254c..ba37dbc5 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -86,10 +86,13 @@ public void validateWrongFlushSize() { "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1kb") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") .build()); } @@ -101,11 +104,13 @@ public void validateWrongRetries() { thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), "0") .build()); } @@ -118,10 +123,13 @@ public void validateWrongMaxActions() { thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "-2") + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") .build()); } @@ -133,11 +141,13 @@ public void validateWrongBackoffDelay() { thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "-1s") + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") .build()); } @@ -193,11 +203,13 @@ public void validatePrimaryKeyOnIllegalColumn() { UniqueConstraint.primaryKey( "name", Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") .withOption( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "1s") + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") .build()); } @@ -210,11 +222,12 @@ public void validateWrongCredential() { "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); sinkFactory.createDynamicTableSink( context() - .withOption(ElasticsearchOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), "http://localhost:1234") - .withOption(ElasticsearchOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchOptions.PASSWORD_OPTION.key(), "") + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") .build()); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 5efaba96..cb025a60 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -117,13 +117,15 @@ public void testWritingDocuments() throws Exception { context() .withSchema(schema) .withOption( - ElasticsearchOptions.INDEX_OPTION.key(), + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), index) .withOption( - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), elasticsearchContainer.getHttpHostAddress()) .withOption( - ElasticsearchOptions + ElasticsearchConnectorOptions .FLUSH_ON_CHECKPOINT_OPTION .key(), "false") @@ -172,14 +174,16 @@ public void testWritingDocumentsFromTableApi() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment @@ -228,14 +232,16 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment @@ -316,14 +322,16 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { + "WITH (\n" + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( - "'%s'='%s',\n", ElasticsearchOptions.INDEX_OPTION.key(), index) + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s',\n", - ElasticsearchOptions.HOSTS_OPTION.key(), + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), elasticsearchContainer.getHttpHostAddress()) + String.format( "'%s'='%s'\n", - ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false") + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + ")"); tableEnvironment diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index 3f3b640d..54c75741 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -95,10 +95,11 @@ public void testBuilder() { public void testDefaultConfig() { final TableSchema schema = createTestSchema(); Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); BuilderProvider provider = new BuilderProvider(); final Elasticsearch7DynamicSink testSink = @@ -125,12 +126,13 @@ public void testDefaultConfig() { public void testAuthConfig() { final TableSchema schema = createTestSchema(); Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchOptions.PASSWORD_OPTION.key(), PASSWORD); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); BuilderProvider provider = new BuilderProvider(); final Elasticsearch7DynamicSink testSink = @@ -157,23 +159,30 @@ public void testAuthConfig() { private Configuration getConfig() { Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); configuration.setString( - ElasticsearchOptions.HOSTS_OPTION.key(), SCHEMA + "://" + HOSTNAME + ":" + PORT); + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); configuration.setString( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); configuration.setString( - ElasticsearchOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString(ElasticsearchOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString(ElasticsearchOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString(ElasticsearchOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); configuration.setString( - ElasticsearchOptions.FAILURE_HANDLER_OPTION.key(), + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), DummyFailureHandler.class.getName()); - configuration.setString(ElasticsearchOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); return configuration; } From 25914cca29bc4341a4cf41a4762fffb99a4c55fc Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Jun 2021 22:03:28 +0200 Subject: [PATCH 139/207] [FLINK-18783] Load Akka with separate classloader --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index ea25c68e..07817acc 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -94,7 +94,7 @@ under the License. org.apache.flink - flink-runtime_${scala.binary.version} + flink-runtime ${project.version} test-jar test diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index b8c89dde..6491887f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -112,7 +112,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index ecac13d1..0141d697 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -110,7 +110,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test From 48994e5967698ec47389c4139aa63df1e87127f4 Mon Sep 17 00:00:00 2001 From: Piotr Nowojski Date: Wed, 14 Jul 2021 09:10:37 +0200 Subject: [PATCH 140/207] Revert "[FLINK-18783] Load Akka with separate classloader" This reverts commit 3ff411d0205c35a3e1a8da822bf6b7f2f5409609. --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 07817acc..ea25c68e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -94,7 +94,7 @@ under the License. org.apache.flink - flink-runtime + flink-runtime_${scala.binary.version} ${project.version} test-jar test diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 6491887f..b8c89dde 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -112,7 +112,7 @@ under the License. org.apache.flink - flink-statebackend-changelog + flink-statebackend-changelog_${scala.binary.version} ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 0141d697..ecac13d1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -110,7 +110,7 @@ under the License. org.apache.flink - flink-statebackend-changelog + flink-statebackend-changelog_${scala.binary.version} ${project.version} test From adf6fc7261b5451e4b7b98d994c68c7adb07dc1d Mon Sep 17 00:00:00 2001 From: Mika Date: Fri, 16 Jul 2021 20:40:32 +0200 Subject: [PATCH 141/207] [FLINK-22386][tests] Cache docker images --- .../elasticsearch/table/Elasticsearch6DynamicSinkITCase.java | 5 ++--- .../connectors/elasticsearch6/ElasticsearchSinkITCase.java | 5 ++--- .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 5 ++--- .../connectors/elasticsearch7/ElasticsearchSinkITCase.java | 5 ++--- 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index e9795015..b7090c51 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; @@ -68,9 +69,7 @@ public class Elasticsearch6DynamicSinkITCase { @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("6.3.1")); + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); @SuppressWarnings("deprecation") protected final Client getClient() { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 76a70203..0637ca52 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.util.DockerImageVersions; import org.apache.http.HttpHost; import org.elasticsearch.client.Client; @@ -43,9 +44,7 @@ public class ElasticsearchSinkITCase @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("6.3.1")); + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); @Override protected String getClusterName() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index cb025a60..18c0b3e1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -36,6 +36,7 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; @@ -68,9 +69,7 @@ public class Elasticsearch7DynamicSinkITCase { @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("7.5.1")); + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); @SuppressWarnings("deprecation") protected final Client getClient() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index b651e6a2..65332a86 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.util.DockerImageVersions; import org.apache.http.HttpHost; import org.elasticsearch.client.Client; @@ -43,9 +44,7 @@ public class ElasticsearchSinkITCase @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer( - DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch-oss") - .withTag("7.5.1")); + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); @Override protected String getClusterName() { From 22be0e791924ecaedbc84b4e5f24d356a7fc604d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Jun 2021 22:03:28 +0200 Subject: [PATCH 142/207] [FLINK-18783] Load Akka with separate classloader --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index ea25c68e..07817acc 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -94,7 +94,7 @@ under the License. org.apache.flink - flink-runtime_${scala.binary.version} + flink-runtime ${project.version} test-jar test diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index b8c89dde..6491887f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -112,7 +112,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index ecac13d1..0141d697 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -110,7 +110,7 @@ under the License. org.apache.flink - flink-statebackend-changelog_${scala.binary.version} + flink-statebackend-changelog ${project.version} test From 14a936d06af00e5b9515129799bc57b8877aeb76 Mon Sep 17 00:00:00 2001 From: martijnvisser Date: Fri, 23 Jul 2021 13:31:19 +0200 Subject: [PATCH 143/207] [FLINK-23438] Bump httpclient from 4.5.3 and 4.5.9 to 4.5.13 + Bump httpcore from 4.4.6 and 4.5.11 to 4.4.14 --- .../src/main/resources/META-INF/NOTICE | 4 ++-- .../src/main/resources/META-INF/NOTICE | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index ff2a4bbd..4bc2241c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -15,8 +15,8 @@ This project bundles the following dependencies under the Apache Software Licens - commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 -- org.apache.httpcomponents:httpclient:4.5.3 -- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 - org.apache.httpcomponents:httpcore-nio:4.4.5 - org.apache.lucene:lucene-analyzers-common:7.3.1 - org.apache.lucene:lucene-backward-codecs:7.3.1 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index e3e63b1e..d14c8181 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -17,8 +17,8 @@ This project bundles the following dependencies under the Apache Software Licens - commons-codec:commons-codec:1.13 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.4 -- org.apache.httpcomponents:httpclient:4.5.3 -- org.apache.httpcomponents:httpcore:4.4.6 +- org.apache.httpcomponents:httpclient:4.5.13 +- org.apache.httpcomponents:httpcore:4.4.14 - org.apache.httpcomponents:httpcore-nio:4.4.12 - org.apache.lucene:lucene-analyzers-common:8.3.0 - org.apache.lucene:lucene-backward-codecs:8.3.0 From f4564441134419ea8861c80193a7d11c09b0685e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ingo=20B=C3=BCrk?= Date: Thu, 29 Jul 2021 10:08:54 +0200 Subject: [PATCH 144/207] [FLINK-25313][connector-elasticsearch] Remove legacy Elasticseach connector Use the new connector via option 'connector' = 'elasticsearch-7'. --- .../ElasticsearchUpsertTableSinkBase.java | 523 ------------------ ...asticsearchUpsertTableSinkFactoryBase.java | 383 ------------- .../table/descriptors/Elasticsearch.java | 325 ----------- .../descriptors/ElasticsearchValidator.java | 200 ------- ...csearchUpsertTableSinkFactoryTestBase.java | 241 -------- .../table/descriptors/ElasticsearchTest.java | 159 ------ .../Elasticsearch6UpsertTableSink.java | 265 --------- .../Elasticsearch6UpsertTableSinkFactory.java | 75 --- ....apache.flink.table.factories.TableFactory | 16 - ...sticsearch6UpsertTableSinkFactoryTest.java | 238 -------- .../Elasticsearch7UpsertTableSink.java | 280 ---------- .../Elasticsearch7UpsertTableSinkFactory.java | 74 --- ....apache.flink.table.factories.TableFactory | 16 - ...sticsearch7UpsertTableSinkFactoryTest.java | 237 -------- 14 files changed, 3032 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java deleted file mode 100644 index 5d47dacb..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkBase.java +++ /dev/null @@ -1,523 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.table.utils.TableConnectorUtils; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.common.xcontent.XContentType; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** A version-agnostic Elasticsearch {@link UpsertStreamTableSink}. */ -@Internal -public abstract class ElasticsearchUpsertTableSinkBase implements UpsertStreamTableSink { - - /** Flag that indicates that only inserts are accepted. */ - private final boolean isAppendOnly; - - /** Schema of the table. */ - private final TableSchema schema; - - /** Version-agnostic hosts configuration. */ - private final List hosts; - - /** Default index for all requests. */ - private final String index; - - /** Default document type for all requests. */ - private final String docType; - - /** Delimiter for composite keys. */ - private final String keyDelimiter; - - /** String literal for null keys. */ - private final String keyNullLiteral; - - /** Serialization schema used for the document. */ - private final SerializationSchema serializationSchema; - - /** Content type describing the serialization schema. */ - private final XContentType contentType; - - /** Failure handler for failing {@link ActionRequest}s. */ - private final ActionRequestFailureHandler failureHandler; - - /** - * Map of optional configuration parameters for the Elasticsearch sink. The config is internal - * and can change at any time. - */ - private final Map sinkOptions; - - /** Version-agnostic creation of {@link ActionRequest}s. */ - private final RequestFactory requestFactory; - - /** Key field indices determined by the query. */ - private int[] keyFieldIndices = new int[0]; - - public ElasticsearchUpsertTableSinkBase( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - this.isAppendOnly = isAppendOnly; - this.schema = TableSchemaUtils.checkOnlyPhysicalColumns(schema); - this.hosts = Preconditions.checkNotNull(hosts); - this.index = Preconditions.checkNotNull(index); - this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); - this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); - this.docType = Preconditions.checkNotNull(docType); - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.failureHandler = Preconditions.checkNotNull(failureHandler); - this.sinkOptions = Preconditions.checkNotNull(sinkOptions); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - } - - @Override - public void setKeyFields(String[] keyNames) { - if (keyNames == null) { - this.keyFieldIndices = new int[0]; - return; - } - - final String[] fieldNames = getFieldNames(); - final int[] keyFieldIndices = new int[keyNames.length]; - for (int i = 0; i < keyNames.length; i++) { - keyFieldIndices[i] = -1; - for (int j = 0; j < fieldNames.length; j++) { - if (keyNames[i].equals(fieldNames[j])) { - keyFieldIndices[i] = j; - break; - } - } - if (keyFieldIndices[i] == -1) { - throw new RuntimeException("Invalid key fields: " + Arrays.toString(keyNames)); - } - } - - validateKeyTypes(keyFieldIndices); - - this.keyFieldIndices = keyFieldIndices; - } - - @Override - public void setIsAppendOnly(Boolean isAppendOnly) { - if (this.isAppendOnly && !isAppendOnly) { - throw new ValidationException( - "The given query is not supported by this sink because the sink is configured to " - + "operate in append mode only. Thus, it only support insertions (no queries " - + "with updating results)."); - } - } - - @Override - public TypeInformation getRecordType() { - return schema.toRowType(); - } - - @Override - public DataStreamSink consumeDataStream(DataStream> dataStream) { - final ElasticsearchUpsertSinkFunction upsertFunction = - new ElasticsearchUpsertSinkFunction( - IndexGeneratorFactory.createIndexGenerator(index, schema), - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - requestFactory, - keyFieldIndices); - final SinkFunction> sinkFunction = - createSinkFunction(hosts, failureHandler, sinkOptions, upsertFunction); - return dataStream - .addSink(sinkFunction) - .setParallelism(dataStream.getParallelism()) - .name(TableConnectorUtils.generateRuntimeName(this.getClass(), getFieldNames())); - } - - @Override - public TypeInformation> getOutputType() { - return Types.TUPLE(Types.BOOLEAN, getRecordType()); - } - - @Override - public String[] getFieldNames() { - return schema.getFieldNames(); - } - - @Override - public TypeInformation[] getFieldTypes() { - return schema.getFieldTypes(); - } - - @Override - public TableSink> configure( - String[] fieldNames, TypeInformation[] fieldTypes) { - if (!Arrays.equals(getFieldNames(), fieldNames) - || !Arrays.equals(getFieldTypes(), fieldTypes)) { - throw new ValidationException( - "Reconfiguration with different fields is not allowed. " - + "Expected: " - + Arrays.toString(getFieldNames()) - + " / " - + Arrays.toString(getFieldTypes()) - + ". " - + "But was: " - + Arrays.toString(fieldNames) - + " / " - + Arrays.toString(fieldTypes)); - } - return copy( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - requestFactory); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ElasticsearchUpsertTableSinkBase that = (ElasticsearchUpsertTableSinkBase) o; - return Objects.equals(isAppendOnly, that.isAppendOnly) - && Objects.equals(schema, that.schema) - && Objects.equals(hosts, that.hosts) - && Objects.equals(index, that.index) - && Objects.equals(docType, that.docType) - && Objects.equals(keyDelimiter, that.keyDelimiter) - && Objects.equals(keyNullLiteral, that.keyNullLiteral) - && Objects.equals(serializationSchema, that.serializationSchema) - && Objects.equals(contentType, that.contentType) - && Objects.equals(failureHandler, that.failureHandler) - && Objects.equals(sinkOptions, that.sinkOptions); - } - - @Override - public int hashCode() { - return Objects.hash( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // For version-specific implementations - // -------------------------------------------------------------------------------------------- - - protected abstract ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory); - - protected abstract SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertFunction); - - // -------------------------------------------------------------------------------------------- - // Helper methods - // -------------------------------------------------------------------------------------------- - - /** Validate the types that are used for conversion to string. */ - private void validateKeyTypes(int[] keyFieldIndices) { - final TypeInformation[] types = getFieldTypes(); - for (int keyFieldIndex : keyFieldIndices) { - final TypeInformation type = types[keyFieldIndex]; - if (!type.isKeyType()) { - throw new ValidationException( - "Only simple types that can be safely converted into a string representation " - + "can be used as keys. But was: " - + type); - } - } - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** Keys for optional parameterization of the sink. */ - public enum SinkOption { - DISABLE_FLUSH_ON_CHECKPOINT, - BULK_FLUSH_MAX_ACTIONS, - BULK_FLUSH_MAX_SIZE, - BULK_FLUSH_INTERVAL, - BULK_FLUSH_BACKOFF_ENABLED, - BULK_FLUSH_BACKOFF_TYPE, - BULK_FLUSH_BACKOFF_RETRIES, - BULK_FLUSH_BACKOFF_DELAY, - REST_MAX_RETRY_TIMEOUT, - REST_PATH_PREFIX - } - - /** Entity for describing a host of Elasticsearch. */ - public static class Host { - public final String hostname; - public final int port; - public final String protocol; - - public Host(String hostname, int port, String protocol) { - this.hostname = hostname; - this.port = port; - this.protocol = protocol; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Host host = (Host) o; - return port == host.port - && Objects.equals(hostname, host.hostname) - && Objects.equals(protocol, host.protocol); - } - - @Override - public int hashCode() { - return Objects.hash(hostname, port, protocol); - } - - @Override - public String toString() { - return protocol + "://" + hostname + ":" + port; - } - } - - /** For version-agnostic creating of {@link ActionRequest}s. */ - public interface RequestFactory extends Serializable { - - /** - * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field - * has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document); - - /** - * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field - * has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - IndexRequest createIndexRequest( - String index, String docType, XContentType contentType, byte[] document); - - /** - * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field - * has been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - DeleteRequest createDeleteRequest(String index, String docType, String key); - } - - /** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ - public static class ElasticsearchUpsertSinkFunction - implements ElasticsearchSinkFunction> { - - private final IndexGenerator indexGenerator; - private final String docType; - private final String keyDelimiter; - private final String keyNullLiteral; - private final SerializationSchema serializationSchema; - private final XContentType contentType; - private final RequestFactory requestFactory; - private final int[] keyFieldIndices; - - public ElasticsearchUpsertSinkFunction( - IndexGenerator indexGenerator, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - RequestFactory requestFactory, - int[] keyFieldIndices) { - - this.indexGenerator = Preconditions.checkNotNull(indexGenerator); - this.docType = Preconditions.checkNotNull(docType); - this.keyDelimiter = Preconditions.checkNotNull(keyDelimiter); - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.keyFieldIndices = Preconditions.checkNotNull(keyFieldIndices); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - this.keyNullLiteral = Preconditions.checkNotNull(keyNullLiteral); - } - - @Override - public void open() { - indexGenerator.open(); - } - - @Override - public void process( - Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) { - - final String formattedIndex = indexGenerator.generate(element.f1); - if (element.f0) { - processUpsert(element.f1, indexer, formattedIndex); - } else { - processDelete(element.f1, indexer, formattedIndex); - } - } - - private void processUpsert(Row row, RequestIndexer indexer, String formattedIndex) { - final byte[] document = serializationSchema.serialize(row); - if (keyFieldIndices.length == 0) { - final IndexRequest indexRequest = - requestFactory.createIndexRequest( - formattedIndex, docType, contentType, document); - indexer.add(indexRequest); - } else { - final String key = createKey(row); - final UpdateRequest updateRequest = - requestFactory.createUpdateRequest( - formattedIndex, docType, key, contentType, document); - indexer.add(updateRequest); - } - } - - private void processDelete(Row row, RequestIndexer indexer, String formattedIndex) { - final String key = createKey(row); - final DeleteRequest deleteRequest = - requestFactory.createDeleteRequest(formattedIndex, docType, key); - indexer.add(deleteRequest); - } - - private String createKey(Row row) { - final StringBuilder builder = new StringBuilder(); - for (int i = 0; i < keyFieldIndices.length; i++) { - final int keyFieldIndex = keyFieldIndices[i]; - if (i > 0) { - builder.append(keyDelimiter); - } - final Object value = row.getField(keyFieldIndex); - if (value == null) { - builder.append(keyNullLiteral); - } else { - builder.append(value.toString()); - } - } - return builder.toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ElasticsearchUpsertSinkFunction that = (ElasticsearchUpsertSinkFunction) o; - return Objects.equals(indexGenerator, that.indexGenerator) - && Objects.equals(docType, that.docType) - && Objects.equals(keyDelimiter, that.keyDelimiter) - && Objects.equals(keyNullLiteral, that.keyNullLiteral) - && Objects.equals(serializationSchema, that.serializationSchema) - && contentType == that.contentType - && Objects.equals(requestFactory, that.requestFactory) - && Arrays.equals(keyFieldIndices, that.keyFieldIndices); - } - - @Override - public int hashCode() { - int result = - Objects.hash( - indexGenerator, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - requestFactory); - result = 31 * result + Arrays.hashCode(keyFieldIndices); - return result; - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java deleted file mode 100644 index db574bb2..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryBase.java +++ /dev/null @@ -1,383 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.apache.flink.table.descriptors.ElasticsearchValidator; -import org.apache.flink.table.descriptors.SchemaValidator; -import org.apache.flink.table.descriptors.StreamTableDescriptorValidator; -import org.apache.flink.table.factories.SerializationSchemaFactory; -import org.apache.flink.table.factories.StreamTableSinkFactory; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.table.sinks.StreamTableSink; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.InstantiationUtil; - -import org.elasticsearch.common.xcontent.XContentType; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_PROPERTY_VERSION; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE; -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_DELAY; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_INTERVAL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_ACTIONS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_SIZE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_PATH_PREFIX; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_DOCUMENT_TYPE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_CLASS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FLUSH_ON_CHECKPOINT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_HOSTNAME; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PORT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS_PROTOCOL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_INDEX; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.validateAndParseHostsString; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT; -import static org.apache.flink.table.descriptors.FormatDescriptorValidator.FORMAT_TYPE; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_DATA_TYPE; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_NAME; -import static org.apache.flink.table.descriptors.Schema.SCHEMA_TYPE; -import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE; -import static org.apache.flink.table.descriptors.StreamTableDescriptorValidator.UPDATE_MODE_VALUE_APPEND; - -/** - * Version-agnostic table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch. - */ -@Internal -public abstract class ElasticsearchUpsertTableSinkFactoryBase - implements StreamTableSinkFactory> { - - private static final String SUPPORTED_FORMAT_TYPE = "json"; - private static final XContentType SUPPORTED_CONTENT_TYPE = XContentType.JSON; - private static final String DEFAULT_KEY_DELIMITER = "_"; - private static final String DEFAULT_KEY_NULL_LITERAL = "null"; - private static final String DEFAULT_FAILURE_HANDLER = CONNECTOR_FAILURE_HANDLER_VALUE_FAIL; - - @Override - public Map requiredContext() { - final Map context = new HashMap<>(); - context.put(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH); - context.put(CONNECTOR_VERSION, elasticsearchVersion()); - context.put(CONNECTOR_PROPERTY_VERSION, "1"); - return context; - } - - @Override - public List supportedProperties() { - final List properties = new ArrayList<>(); - - // streaming properties - properties.add(UPDATE_MODE); - - // Elasticsearch - properties.add(CONNECTOR_HOSTS); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_HOSTNAME); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PORT); - properties.add(CONNECTOR_HOSTS + ".#." + CONNECTOR_HOSTS_PROTOCOL); - properties.add(CONNECTOR_INDEX); - properties.add(CONNECTOR_DOCUMENT_TYPE); - properties.add(CONNECTOR_KEY_DELIMITER); - properties.add(CONNECTOR_KEY_NULL_LITERAL); - properties.add(CONNECTOR_FAILURE_HANDLER); - properties.add(CONNECTOR_FAILURE_HANDLER_CLASS); - properties.add(CONNECTOR_FLUSH_ON_CHECKPOINT); - properties.add(CONNECTOR_BULK_FLUSH_MAX_ACTIONS); - properties.add(CONNECTOR_BULK_FLUSH_MAX_SIZE); - properties.add(CONNECTOR_BULK_FLUSH_INTERVAL); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES); - properties.add(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY); - properties.add(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT); - properties.add(CONNECTOR_CONNECTION_PATH_PREFIX); - - // schema - properties.add(SCHEMA + ".#." + SCHEMA_DATA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_TYPE); - properties.add(SCHEMA + ".#." + SCHEMA_NAME); - // computed column - properties.add(SCHEMA + ".#." + EXPR); - - // watermark - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_ROWTIME); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_EXPR); - properties.add(SCHEMA + "." + WATERMARK + ".#." + WATERMARK_STRATEGY_DATA_TYPE); - - // table constraint - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_NAME); - properties.add(SCHEMA + "." + DescriptorProperties.PRIMARY_KEY_COLUMNS); - - // comment - properties.add(COMMENT); - - // format wildcard - properties.add(FORMAT + ".*"); - - return properties; - } - - @Override - public StreamTableSink> createStreamTableSink( - Map properties) { - final DescriptorProperties descriptorProperties = getValidatedProperties(properties); - - return createElasticsearchUpsertTableSink( - descriptorProperties.isValue(UPDATE_MODE, UPDATE_MODE_VALUE_APPEND), - TableSchemaUtils.getPhysicalSchema(descriptorProperties.getTableSchema(SCHEMA)), - getHosts(descriptorProperties), - descriptorProperties.getString(CONNECTOR_INDEX), - descriptorProperties.getString(CONNECTOR_DOCUMENT_TYPE), - descriptorProperties - .getOptionalString(CONNECTOR_KEY_DELIMITER) - .orElse(DEFAULT_KEY_DELIMITER), - descriptorProperties - .getOptionalString(CONNECTOR_KEY_NULL_LITERAL) - .orElse(DEFAULT_KEY_NULL_LITERAL), - getSerializationSchema(properties), - SUPPORTED_CONTENT_TYPE, - getFailureHandler(descriptorProperties), - getSinkOptions(descriptorProperties)); - } - - // -------------------------------------------------------------------------------------------- - // For version-specific factories - // -------------------------------------------------------------------------------------------- - - protected abstract String elasticsearchVersion(); - - protected abstract ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions); - - // -------------------------------------------------------------------------------------------- - // Helper methods - // -------------------------------------------------------------------------------------------- - - private DescriptorProperties getValidatedProperties(Map properties) { - final DescriptorProperties descriptorProperties = new DescriptorProperties(true); - descriptorProperties.putProperties(properties); - - new StreamTableDescriptorValidator(true, false, true).validate(descriptorProperties); - new SchemaValidator(true, false, false).validate(descriptorProperties); - new ElasticsearchValidator().validate(descriptorProperties); - - return descriptorProperties; - } - - private List getHosts(DescriptorProperties descriptorProperties) { - if (descriptorProperties.containsKey(CONNECTOR_HOSTS)) { - return validateAndParseHostsString(descriptorProperties); - } else { - final List> hosts = - descriptorProperties.getFixedIndexedProperties( - CONNECTOR_HOSTS, - Arrays.asList( - CONNECTOR_HOSTS_HOSTNAME, - CONNECTOR_HOSTS_PORT, - CONNECTOR_HOSTS_PROTOCOL)); - return hosts.stream() - .map( - host -> - new Host( - descriptorProperties.getString( - host.get(CONNECTOR_HOSTS_HOSTNAME)), - descriptorProperties.getInt( - host.get(CONNECTOR_HOSTS_PORT)), - descriptorProperties.getString( - host.get(CONNECTOR_HOSTS_PROTOCOL)))) - .collect(Collectors.toList()); - } - } - - private SerializationSchema getSerializationSchema(Map properties) { - final String formatType = properties.get(FORMAT_TYPE); - // we could have added this check to the table factory context - // but this approach allows to throw more helpful error messages - // if the supported format has not been added - if (formatType == null || !formatType.equals(SUPPORTED_FORMAT_TYPE)) { - throw new ValidationException( - "The Elasticsearch sink requires a '" + SUPPORTED_FORMAT_TYPE + "' format."); - } - - @SuppressWarnings("unchecked") - final SerializationSchemaFactory formatFactory = - TableFactoryService.find( - SerializationSchemaFactory.class, - properties, - this.getClass().getClassLoader()); - return formatFactory.createSerializationSchema(properties); - } - - private ActionRequestFailureHandler getFailureHandler( - DescriptorProperties descriptorProperties) { - final String failureHandler = - descriptorProperties - .getOptionalString(CONNECTOR_FAILURE_HANDLER) - .orElse(DEFAULT_FAILURE_HANDLER); - switch (failureHandler) { - case CONNECTOR_FAILURE_HANDLER_VALUE_FAIL: - return new NoOpFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE: - return new IgnoringFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_RETRY: - return new RetryRejectedExecutionFailureHandler(); - case CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM: - final Class clazz = - descriptorProperties.getClass( - CONNECTOR_FAILURE_HANDLER_CLASS, ActionRequestFailureHandler.class); - return InstantiationUtil.instantiate(clazz); - default: - throw new IllegalArgumentException("Unknown failure handler."); - } - } - - private Map getSinkOptions(DescriptorProperties descriptorProperties) { - final Map options = new HashMap<>(); - - descriptorProperties - .getOptionalBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT) - .ifPresent( - v -> - options.put( - SinkOption.DISABLE_FLUSH_ON_CHECKPOINT, - String.valueOf(!v))); - - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_BULK_FLUSH_MAX_ACTIONS, - SinkOption.BULK_FLUSH_MAX_ACTIONS); - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_BULK_FLUSH_MAX_SIZE, - SinkOption.BULK_FLUSH_MAX_SIZE); - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_BULK_FLUSH_INTERVAL, - SinkOption.BULK_FLUSH_INTERVAL); - - descriptorProperties - .getOptionalString(CONNECTOR_BULK_FLUSH_BACKOFF_TYPE) - .ifPresent( - v -> { - options.put( - SinkOption.BULK_FLUSH_BACKOFF_ENABLED, - String.valueOf( - !v.equals( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED))); - switch (v) { - case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT: - options.put( - SinkOption.BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchSinkBase.FlushBackoffType.CONSTANT - .toString()); - break; - case CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL: - options.put( - SinkOption.BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL - .toString()); - break; - default: - throw new IllegalArgumentException("Unknown backoff type."); - } - }); - - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, - SinkOption.BULK_FLUSH_BACKOFF_RETRIES); - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, - SinkOption.BULK_FLUSH_BACKOFF_DELAY); - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, - SinkOption.REST_MAX_RETRY_TIMEOUT); - mapSinkOption( - descriptorProperties, - options, - CONNECTOR_CONNECTION_PATH_PREFIX, - SinkOption.REST_PATH_PREFIX); - - return options; - } - - private void mapSinkOption( - DescriptorProperties descriptorProperties, - Map options, - String fromKey, - SinkOption toKey) { - descriptorProperties.getOptionalString(fromKey).ifPresent(v -> options.put(toKey, v)); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java deleted file mode 100644 index 710fe164..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/Elasticsearch.java +++ /dev/null @@ -1,325 +0,0 @@ -/* - * 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.table.descriptors; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.util.Preconditions; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_VERSION; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_DELAY; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_INTERVAL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_ACTIONS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_BULK_FLUSH_MAX_SIZE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_CONNECTION_PATH_PREFIX; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_DOCUMENT_TYPE; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_CLASS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_FLUSH_ON_CHECKPOINT; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_INDEX; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_DELIMITER; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_KEY_NULL_LITERAL; -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_TYPE_VALUE_ELASTICSEARCH; - -/** Connector descriptor for the Elasticsearch search engine. */ -@PublicEvolving -public class Elasticsearch extends ConnectorDescriptor { - - private DescriptorProperties internalProperties = new DescriptorProperties(true); - private List hosts = new ArrayList<>(); - - /** Connector descriptor for the Elasticsearch search engine. */ - public Elasticsearch() { - super(CONNECTOR_TYPE_VALUE_ELASTICSEARCH, 1, true); - } - - /** - * Sets the Elasticsearch version to be used. Required. - * - * @param version Elasticsearch version. E.g., "6". - */ - public Elasticsearch version(String version) { - internalProperties.putString(CONNECTOR_VERSION, version); - return this; - } - - /** - * Adds an Elasticsearch host to connect to. Required. - * - *

Multiple hosts can be declared by calling this method multiple times. - * - * @param hostname connection hostname - * @param port connection port - * @param protocol connection protocol; e.g. "http" - */ - public Elasticsearch host(String hostname, int port, String protocol) { - final Host host = - new Host( - Preconditions.checkNotNull(hostname), - port, - Preconditions.checkNotNull(protocol)); - hosts.add(host); - return this; - } - - /** - * Declares the Elasticsearch index for every record. Required. - * - * @param index Elasticsearch index - */ - public Elasticsearch index(String index) { - internalProperties.putString(CONNECTOR_INDEX, index); - return this; - } - - /** - * Declares the Elasticsearch document type for every record. Required. - * - * @param documentType Elasticsearch document type - */ - public Elasticsearch documentType(String documentType) { - internalProperties.putString(CONNECTOR_DOCUMENT_TYPE, documentType); - return this; - } - - /** - * Sets a custom key delimiter in case the Elasticsearch ID needs to be constructed from - * multiple fields. Optional. - * - * @param keyDelimiter key delimiter; e.g., "$" would result in IDs "KEY1$KEY2$KEY3" - */ - public Elasticsearch keyDelimiter(String keyDelimiter) { - internalProperties.putString(CONNECTOR_KEY_DELIMITER, keyDelimiter); - return this; - } - - /** - * Sets a custom representation for null fields in keys. Optional. - * - * @param keyNullLiteral key null literal string; e.g. "N/A" would result in IDs "KEY1_N/A_KEY3" - */ - public Elasticsearch keyNullLiteral(String keyNullLiteral) { - internalProperties.putString(CONNECTOR_KEY_NULL_LITERAL, keyNullLiteral); - return this; - } - - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy throws an exception if a request fails and thus causes a job failure. - */ - public Elasticsearch failureHandlerFail() { - internalProperties.putString( - CONNECTOR_FAILURE_HANDLER, - ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_FAIL); - return this; - } - - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy ignores failures and drops the request. - */ - public Elasticsearch failureHandlerIgnore() { - internalProperties.putString( - CONNECTOR_FAILURE_HANDLER, - ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE); - return this; - } - - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy re-adds requests that have failed due to queue capacity saturation. - */ - public Elasticsearch failureHandlerRetryRejected() { - internalProperties.putString( - CONNECTOR_FAILURE_HANDLER, - ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_RETRY); - return this; - } - - /** - * Configures a failure handling strategy in case a request to Elasticsearch fails. - * - *

This strategy allows for custom failure handling using a {@link - * ActionRequestFailureHandler}. - */ - public Elasticsearch failureHandlerCustom( - Class failureHandlerClass) { - internalProperties.putString( - CONNECTOR_FAILURE_HANDLER, - ElasticsearchValidator.CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM); - internalProperties.putClass(CONNECTOR_FAILURE_HANDLER_CLASS, failureHandlerClass); - return this; - } - - /** - * Disables flushing on checkpoint. When disabled, a sink will not wait for all pending action - * requests to be acknowledged by Elasticsearch on checkpoints. - * - *

Note: If flushing on checkpoint is disabled, a Elasticsearch sink does NOT provide any - * strong guarantees for at-least-once delivery of action requests. - */ - public Elasticsearch disableFlushOnCheckpoint() { - internalProperties.putBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, false); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum number of actions to buffer for each bulk request. - * - * @param maxActions the maximum number of actions to buffer per bulk request. - */ - public Elasticsearch bulkFlushMaxActions(int maxActions) { - internalProperties.putInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, maxActions); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum size of buffered actions per bulk request (using the syntax of {@link - * MemorySize}). - */ - public Elasticsearch bulkFlushMaxSize(String maxSize) { - internalProperties.putMemorySize( - CONNECTOR_BULK_FLUSH_MAX_SIZE, - MemorySize.parse(maxSize, MemorySize.MemoryUnit.BYTES)); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the bulk flush interval (in milliseconds). - * - * @param interval bulk flush interval (in milliseconds). - */ - public Elasticsearch bulkFlushInterval(long interval) { - internalProperties.putLong(CONNECTOR_BULK_FLUSH_INTERVAL, interval); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets a constant backoff type to use when flushing bulk requests. - */ - public Elasticsearch bulkFlushBackoffConstant() { - internalProperties.putString( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets an exponential backoff type to use when flushing bulk requests. - */ - public Elasticsearch bulkFlushBackoffExponential() { - internalProperties.putString( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - ElasticsearchValidator.CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. - * - *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} - * or {@link #bulkFlushBackoffExponential()}). - * - * @param maxRetries the maximum number of retries. - */ - public Elasticsearch bulkFlushBackoffMaxRetries(int maxRetries) { - internalProperties.putInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, maxRetries); - return this; - } - - /** - * Configures how to buffer elements before sending them in bulk to the cluster for efficiency. - * - *

Sets the amount of delay between each backoff attempt when flushing bulk requests (in - * milliseconds). - * - *

Make sure to enable backoff by selecting a strategy ({@link #bulkFlushBackoffConstant()} - * or {@link #bulkFlushBackoffExponential()}). - * - * @param delay delay between each backoff attempt (in milliseconds). - */ - public Elasticsearch bulkFlushBackoffDelay(long delay) { - internalProperties.putLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, delay); - return this; - } - - /** - * Sets connection properties to be used during REST communication to Elasticsearch. - * - *

Sets the maximum timeout (in milliseconds) in case of multiple retries of the same - * request. - * - * @param maxRetryTimeout maximum timeout (in milliseconds) - */ - public Elasticsearch connectionMaxRetryTimeout(int maxRetryTimeout) { - internalProperties.putInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, maxRetryTimeout); - return this; - } - - /** - * Sets connection properties to be used during REST communication to Elasticsearch. - * - *

Adds a path prefix to every REST communication. - * - * @param pathPrefix prefix string to be added to every REST communication - */ - public Elasticsearch connectionPathPrefix(String pathPrefix) { - internalProperties.putString(CONNECTOR_CONNECTION_PATH_PREFIX, pathPrefix); - return this; - } - - @Override - protected Map toConnectorProperties() { - final DescriptorProperties properties = new DescriptorProperties(); - properties.putProperties(internalProperties); - - if (hosts.size() > 0) { - properties.putString( - CONNECTOR_HOSTS, - hosts.stream().map(Host::toString).collect(Collectors.joining(";"))); - } - return properties.asMap(); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java deleted file mode 100644 index da82fb7b..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/table/descriptors/ElasticsearchValidator.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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.table.descriptors; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.util.StringUtils; - -import java.net.MalformedURLException; -import java.net.URL; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Consumer; - -import static org.apache.flink.table.descriptors.DescriptorProperties.noValidation; - -/** The validator for {@link Elasticsearch}. */ -@Internal -public class ElasticsearchValidator extends ConnectorDescriptorValidator { - - public static final String CONNECTOR_TYPE_VALUE_ELASTICSEARCH = "elasticsearch"; - public static final String CONNECTOR_VERSION_VALUE_6 = "6"; - public static final String CONNECTOR_VERSION_VALUE_7 = "7"; - public static final String CONNECTOR_HOSTS = "connector.hosts"; - public static final String CONNECTOR_HOSTS_HOSTNAME = "hostname"; - public static final String CONNECTOR_HOSTS_PORT = "port"; - public static final String CONNECTOR_HOSTS_PROTOCOL = "protocol"; - public static final String CONNECTOR_INDEX = "connector.index"; - public static final String CONNECTOR_DOCUMENT_TYPE = "connector.document-type"; - public static final String CONNECTOR_KEY_DELIMITER = "connector.key-delimiter"; - public static final String CONNECTOR_KEY_NULL_LITERAL = "connector.key-null-literal"; - public static final String CONNECTOR_FAILURE_HANDLER = "connector.failure-handler"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_FAIL = "fail"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE = "ignore"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_RETRY = "retry-rejected"; - public static final String CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM = "custom"; - public static final String CONNECTOR_FAILURE_HANDLER_CLASS = "connector.failure-handler-class"; - public static final String CONNECTOR_FLUSH_ON_CHECKPOINT = "connector.flush-on-checkpoint"; - public static final String CONNECTOR_BULK_FLUSH_MAX_ACTIONS = - "connector.bulk-flush.max-actions"; - public static final String CONNECTOR_BULK_FLUSH_MAX_SIZE = "connector.bulk-flush.max-size"; - public static final String CONNECTOR_BULK_FLUSH_INTERVAL = "connector.bulk-flush.interval"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE = - "connector.bulk-flush.backoff.type"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED = "disabled"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT = "constant"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL = "exponential"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES = - "connector.bulk-flush.backoff.max-retries"; - public static final String CONNECTOR_BULK_FLUSH_BACKOFF_DELAY = - "connector.bulk-flush.backoff.delay"; - public static final String CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT = - "connector.connection-max-retry-timeout"; - public static final String CONNECTOR_CONNECTION_PATH_PREFIX = - "connector.connection-path-prefix"; - - @Override - public void validate(DescriptorProperties properties) { - super.validate(properties); - properties.validateValue(CONNECTOR_TYPE, CONNECTOR_TYPE_VALUE_ELASTICSEARCH, false); - validateVersion(properties); - validateHosts(properties); - validateGeneralProperties(properties); - validateFailureHandler(properties); - validateBulkFlush(properties); - validateConnectionProperties(properties); - } - - private void validateVersion(DescriptorProperties properties) { - properties.validateEnumValues( - CONNECTOR_VERSION, - false, - Arrays.asList(CONNECTOR_VERSION_VALUE_6, CONNECTOR_VERSION_VALUE_7)); - } - - private void validateHosts(DescriptorProperties properties) { - if (properties.containsKey(CONNECTOR_HOSTS)) { - validateAndParseHostsString(properties); - } else { - final Map> hostsValidators = new HashMap<>(); - hostsValidators.put( - CONNECTOR_HOSTS_HOSTNAME, (key) -> properties.validateString(key, false, 1)); - hostsValidators.put( - CONNECTOR_HOSTS_PORT, (key) -> properties.validateInt(key, false, 0, 65535)); - hostsValidators.put( - CONNECTOR_HOSTS_PROTOCOL, (key) -> properties.validateString(key, false, 1)); - - properties.validateFixedIndexedProperties(CONNECTOR_HOSTS, false, hostsValidators); - } - } - - private void validateGeneralProperties(DescriptorProperties properties) { - properties.validateString(CONNECTOR_INDEX, false, 1); - properties.validateString(CONNECTOR_DOCUMENT_TYPE, false, 1); - properties.validateString(CONNECTOR_KEY_DELIMITER, true); - properties.validateString(CONNECTOR_KEY_NULL_LITERAL, true); - } - - private void validateFailureHandler(DescriptorProperties properties) { - final Map> failureHandlerValidators = new HashMap<>(); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_FAIL, noValidation()); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_IGNORE, noValidation()); - failureHandlerValidators.put(CONNECTOR_FAILURE_HANDLER_VALUE_RETRY, noValidation()); - failureHandlerValidators.put( - CONNECTOR_FAILURE_HANDLER_VALUE_CUSTOM, - key -> properties.validateString(CONNECTOR_FAILURE_HANDLER_CLASS, false, 1)); - properties.validateEnum(CONNECTOR_FAILURE_HANDLER, true, failureHandlerValidators); - } - - private void validateBulkFlush(DescriptorProperties properties) { - properties.validateBoolean(CONNECTOR_FLUSH_ON_CHECKPOINT, true); - properties.validateInt(CONNECTOR_BULK_FLUSH_MAX_ACTIONS, true, 1); - properties.validateMemorySize( - CONNECTOR_BULK_FLUSH_MAX_SIZE, true, 1024 * 1024); // only allow MB precision - properties.validateLong(CONNECTOR_BULK_FLUSH_INTERVAL, true, 0); - properties.validateEnumValues( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE, - true, - Arrays.asList( - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_DISABLED, - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_CONSTANT, - CONNECTOR_BULK_FLUSH_BACKOFF_TYPE_VALUE_EXPONENTIAL)); - properties.validateInt(CONNECTOR_BULK_FLUSH_BACKOFF_MAX_RETRIES, true, 1); - properties.validateLong(CONNECTOR_BULK_FLUSH_BACKOFF_DELAY, true, 0); - } - - private void validateConnectionProperties(DescriptorProperties properties) { - properties.validateInt(CONNECTOR_CONNECTION_MAX_RETRY_TIMEOUT, true, 1); - properties.validateString(CONNECTOR_CONNECTION_PATH_PREFIX, true); - } - - /** - * Parse Hosts String to list. - * - *

Hosts String format was given as following: - * - *

-     *     connector.hosts = http://host_name:9092;http://host_name:9093
-     * 
- */ - public static List validateAndParseHostsString( - DescriptorProperties descriptorProperties) { - final List hostList = new ArrayList<>(); - - descriptorProperties.validateString(CONNECTOR_HOSTS, false, 1); - final String hostsStr = descriptorProperties.getString(CONNECTOR_HOSTS); - - final String[] hosts = hostsStr.split(";"); - final String validationExceptionMessage = - "Properties '" - + CONNECTOR_HOSTS - + "' format should " - + "follow the format 'http://host_name:port', but is '" - + hostsStr - + "'."; - - if (hosts.length == 0) { - throw new ValidationException(validationExceptionMessage); - } - for (String host : hosts) { - try { - final URL url = new URL(host); - final String protocol = url.getProtocol(); - final String hostName = url.getHost(); - final int hostPort = url.getPort(); - - if (StringUtils.isNullOrWhitespaceOnly(protocol) - || StringUtils.isNullOrWhitespaceOnly(hostName) - || -1 == hostPort) { - throw new ValidationException(validationExceptionMessage); - } - - hostList.add(new Host(hostName, hostPort, protocol)); - } catch (MalformedURLException e) { - throw new ValidationException(validationExceptionMessage, e); - } - } - return hostList; - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java deleted file mode 100644 index a67953d8..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchUpsertTableSinkFactoryTestBase.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.connectors.elasticsearch; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.formats.json.JsonRowSerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.factories.StreamTableSinkFactory; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; -import org.apache.flink.util.TestLogger; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.common.xcontent.XContentType; -import org.junit.Test; - -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -/** Version-agnostic test base for {@link ElasticsearchUpsertTableSinkFactoryBase}. */ -public abstract class ElasticsearchUpsertTableSinkFactoryTestBase extends TestLogger { - - protected static final String HOSTNAME = "host1"; - protected static final int PORT = 1234; - protected static final String SCHEMA = "https"; - protected static final String INDEX = "MyIndex"; - protected static final String DOC_TYPE = "MyType"; - protected static final String KEY_DELIMITER = "#"; - protected static final String KEY_NULL_LITERAL = ""; - - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - @Test - public void testTableSink() { - // prepare parameters for Elasticsearch table sink - - final TableSchema schema = createTestSchema(); - - final ElasticsearchUpsertTableSinkBase expectedSink = - getExpectedTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions(), - IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); - - // construct table sink using descriptors and table sink factory - final Map elasticSearchProperties = createElasticSearchProperties(); - final TableSink actualSink = - TableFactoryService.find(StreamTableSinkFactory.class, elasticSearchProperties) - .createStreamTableSink(elasticSearchProperties); - - assertEquals(expectedSink, actualSink); - } - - @Test - public void testTableSinkWithLegacyProperties() { - // prepare parameters for Elasticsearch table sink - final TableSchema schema = createTestSchema(); - - final ElasticsearchUpsertTableSinkBase expectedSink = - getExpectedTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions(), - IndexGeneratorFactory.createIndexGenerator(INDEX, schema)); - - // construct table sink using descriptors and table sink factory - final Map elasticSearchProperties = createElasticSearchProperties(); - - final Map legacyPropertiesMap = new HashMap<>(); - legacyPropertiesMap.putAll(elasticSearchProperties); - // use legacy properties - legacyPropertiesMap.remove("connector.hosts"); - - legacyPropertiesMap.put("connector.hosts.0.hostname", "host1"); - legacyPropertiesMap.put("connector.hosts.0.port", "1234"); - legacyPropertiesMap.put("connector.hosts.0.protocol", "https"); - - final TableSink actualSink = - TableFactoryService.find(StreamTableSinkFactory.class, legacyPropertiesMap) - .createStreamTableSink(legacyPropertiesMap); - - assertEquals(expectedSink, actualSink); - } - - protected TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - protected Map createTestSinkOptions() { - final Map sinkOptions = new HashMap<>(); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_ENABLED, "true"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_TYPE, "EXPONENTIAL"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_DELAY, "123"); - sinkOptions.put(SinkOption.BULK_FLUSH_BACKOFF_RETRIES, "3"); - sinkOptions.put(SinkOption.BULK_FLUSH_INTERVAL, "100"); - sinkOptions.put(SinkOption.BULK_FLUSH_MAX_ACTIONS, "1000"); - sinkOptions.put(SinkOption.BULK_FLUSH_MAX_SIZE, "1 mb"); - sinkOptions.put(SinkOption.REST_MAX_RETRY_TIMEOUT, "100"); - sinkOptions.put(SinkOption.REST_PATH_PREFIX, "/myapp"); - return sinkOptions; - } - - protected Map createElasticSearchProperties() { - final Map map = new HashMap<>(); - map.put("connector.bulk-flush.backoff.type", "exponential"); - map.put("connector.bulk-flush.max-size", "1 mb"); - map.put("schema.0.data-type", "BIGINT"); - map.put("schema.1.name", "fruit_name"); - map.put("connector.property-version", "1"); - map.put("connector.bulk-flush.backoff.max-retries", "3"); - map.put("schema.3.data-type", "TIMESTAMP(3)"); - map.put("connector.document-type", "MyType"); - map.put("schema.3.name", "ts"); - map.put("connector.index", "MyIndex"); - map.put("schema.0.name", "key"); - map.put("connector.bulk-flush.backoff.delay", "123"); - map.put("connector.bulk-flush.max-actions", "1000"); - map.put("schema.2.name", "count"); - map.put("update-mode", "upsert"); - map.put( - "connector.failure-handler-class", - ElasticsearchUpsertTableSinkFactoryTestBase.DummyFailureHandler.class.getName()); - map.put("format.type", "json"); - map.put("schema.1.data-type", "VARCHAR(2147483647)"); - map.put("connector.version", getElasticsearchVersion()); - map.put("connector.bulk-flush.interval", "100"); - map.put("schema.2.data-type", "DECIMAL(10, 4)"); - map.put("connector.hosts", "https://host1:1234"); - map.put("connector.failure-handler", "custom"); - map.put("format.property-version", "1"); - map.put("format.derive-schema", "true"); - map.put("connector.type", "elasticsearch"); - map.put("connector.key-null-literal", ""); - map.put("connector.key-delimiter", "#"); - map.put("connector.connection-path-prefix", "/myapp"); - map.put("connector.connection-max-retry-timeout", "100"); - return map; - } - - // -------------------------------------------------------------------------------------------- - // For version-specific tests - // -------------------------------------------------------------------------------------------- - - protected abstract String getElasticsearchVersion(); - - protected abstract ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator); - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** Custom failure handler for testing. */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure( - ActionRequest action, - Throwable failure, - int restStatusCode, - RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return this == o || o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java deleted file mode 100644 index f96aece8..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/table/descriptors/ElasticsearchTest.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.table.descriptors; - -import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; -import org.apache.flink.table.api.ValidationException; - -import org.junit.Test; - -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_HOSTS; - -/** Tests for the {@link Elasticsearch} descriptor. */ -public class ElasticsearchTest extends DescriptorTestBase { - - @Test(expected = ValidationException.class) - public void testMissingIndex() { - removePropertyAndVerify(descriptors().get(0), "connector.index"); - } - - @Test(expected = ValidationException.class) - public void testInvalidFailureHandler() { - addPropertyAndVerify(descriptors().get(0), "connector.failure-handler", "invalid handler"); - } - - @Test(expected = ValidationException.class) - public void testInvalidMemorySize() { - addPropertyAndVerify(descriptors().get(1), "connector.bulk-flush.max-size", "12 bytes"); - } - - @Test(expected = ValidationException.class) - public void testInvalidProtocolInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "localhost:90"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Test(expected = ValidationException.class) - public void testInvalidHostNameInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "http://:90"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Test(expected = ValidationException.class) - public void testInvalidHostPortInHosts() { - final DescriptorProperties descriptorProperties = new DescriptorProperties(); - descriptorProperties.putString(CONNECTOR_HOSTS, "http://localhost"); - ElasticsearchValidator.validateAndParseHostsString(descriptorProperties); - } - - @Override - public List descriptors() { - final Descriptor minimumDesc = - new Elasticsearch() - .version("6") - .host("localhost", 1234, "http") - .index("MyIndex") - .documentType("MyType"); - - final Descriptor maximumDesc = - new Elasticsearch() - .version("6") - .host("host1", 1234, "https") - .host("host2", 1234, "https") - .index("MyIndex") - .documentType("MyType") - .keyDelimiter("#") - .keyNullLiteral("") - .bulkFlushBackoffExponential() - .bulkFlushBackoffDelay(123L) - .bulkFlushBackoffMaxRetries(3) - .bulkFlushInterval(100L) - .bulkFlushMaxActions(1000) - .bulkFlushMaxSize("12 MB") - .failureHandlerRetryRejected() - .connectionMaxRetryTimeout(100) - .connectionPathPrefix("/myapp"); - - final Descriptor customDesc = - new Elasticsearch() - .version("6") - .host("localhost", 1234, "http") - .index("MyIndex") - .documentType("MyType") - .disableFlushOnCheckpoint() - .failureHandlerCustom(NoOpFailureHandler.class); - - return Arrays.asList(minimumDesc, maximumDesc, customDesc); - } - - @Override - public List> properties() { - final Map minimumDesc = new HashMap<>(); - minimumDesc.put("connector.property-version", "1"); - minimumDesc.put("connector.type", "elasticsearch"); - minimumDesc.put("connector.version", "6"); - minimumDesc.put("connector.hosts", "http://localhost:1234"); - minimumDesc.put("connector.index", "MyIndex"); - minimumDesc.put("connector.document-type", "MyType"); - - final Map maximumDesc = new HashMap<>(); - maximumDesc.put("connector.property-version", "1"); - maximumDesc.put("connector.type", "elasticsearch"); - maximumDesc.put("connector.version", "6"); - maximumDesc.put("connector.hosts", "https://host1:1234;https://host2:1234"); - maximumDesc.put("connector.index", "MyIndex"); - maximumDesc.put("connector.document-type", "MyType"); - maximumDesc.put("connector.key-delimiter", "#"); - maximumDesc.put("connector.key-null-literal", ""); - maximumDesc.put("connector.bulk-flush.backoff.type", "exponential"); - maximumDesc.put("connector.bulk-flush.backoff.delay", "123"); - maximumDesc.put("connector.bulk-flush.backoff.max-retries", "3"); - maximumDesc.put("connector.bulk-flush.interval", "100"); - maximumDesc.put("connector.bulk-flush.max-actions", "1000"); - maximumDesc.put("connector.bulk-flush.max-size", "12 mb"); - maximumDesc.put("connector.failure-handler", "retry-rejected"); - maximumDesc.put("connector.connection-max-retry-timeout", "100"); - maximumDesc.put("connector.connection-path-prefix", "/myapp"); - - final Map customDesc = new HashMap<>(); - customDesc.put("connector.property-version", "1"); - customDesc.put("connector.type", "elasticsearch"); - customDesc.put("connector.version", "6"); - customDesc.put("connector.hosts", "http://localhost:1234"); - customDesc.put("connector.index", "MyIndex"); - customDesc.put("connector.document-type", "MyType"); - customDesc.put("connector.flush-on-checkpoint", "false"); - customDesc.put("connector.failure-handler", "custom"); - customDesc.put("connector.failure-handler-class", NoOpFailureHandler.class.getName()); - - return Arrays.asList(minimumDesc, maximumDesc, customDesc); - } - - @Override - public DescriptorValidator validator() { - return new ElasticsearchValidator(); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java deleted file mode 100644 index 1a6a547b..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSink.java +++ /dev/null @@ -1,265 +0,0 @@ -/* - * 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.connectors.elasticsearch6; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.common.xcontent.XContentType; - -import javax.annotation.Nullable; - -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_DELAY; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_ENABLED; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_RETRIES; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_TYPE; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_INTERVAL; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_ACTIONS; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_SIZE; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.DISABLE_FLUSH_ON_CHECKPOINT; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_MAX_RETRY_TIMEOUT; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; - -/** Version-specific upsert table sink for Elasticsearch 6. */ -@Internal -public class Elasticsearch6UpsertTableSink extends ElasticsearchUpsertTableSinkBase { - - @VisibleForTesting - static final RequestFactory UPDATE_REQUEST_FACTORY = new Elasticsearch6RequestFactory(); - - public Elasticsearch6UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @Override - protected ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertSinkFunction) { - - final List httpHosts = - hosts.stream() - .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) - .collect(Collectors.toList()); - - final ElasticsearchSink.Builder> builder = - createBuilder(upsertSinkFunction, httpHosts); - - builder.setFailureHandler(failureHandler); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) - .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) - .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) - .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) - .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) - .ifPresent( - v -> - builder.setBulkFlushBackoffType( - ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) - .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) - .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); - - builder.setRestClientFactory( - new DefaultRestClientFactory( - Optional.ofNullable(sinkOptions.get(REST_MAX_RETRY_TIMEOUT)) - .map(Integer::valueOf) - .orElse(null), - sinkOptions.get(REST_PATH_PREFIX))); - - final ElasticsearchSink> sink = builder.build(); - - Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) - .ifPresent( - v -> { - if (Boolean.valueOf(v)) { - sink.disableFlushOnCheckpoint(); - } - }); - - return sink; - } - - @VisibleForTesting - ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { - return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** Serializable {@link RestClientFactory} used by the sink. */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private Integer maxRetryTimeout; - private String pathPrefix; - - public DefaultRestClientFactory( - @Nullable Integer maxRetryTimeout, @Nullable String pathPrefix) { - this.maxRetryTimeout = maxRetryTimeout; - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (maxRetryTimeout != null) { - restClientBuilder.setMaxRetryTimeoutMillis(maxRetryTimeout); - } - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(maxRetryTimeout, that.maxRetryTimeout) - && Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(maxRetryTimeout, pathPrefix); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the - * sink. - */ - private static class Elasticsearch6RequestFactory implements RequestFactory { - - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, docType, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, String docType, XContentType contentType, byte[] document) { - return new IndexRequest(index, docType).source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, docType, key); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java deleted file mode 100644 index 048a9f13..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactory.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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.connectors.elasticsearch6; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryBase; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.types.Row; - -import org.elasticsearch.common.xcontent.XContentType; - -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_6; - -/** Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 6. */ -@Internal -public class Elasticsearch6UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { - - @Override - protected String elasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_6; - } - - @Override - protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 14c309d1..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +0,0 @@ -# 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. - -org.apache.flink.streaming.connectors.elasticsearch6.Elasticsearch6UpsertTableSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java deleted file mode 100644 index 743040d8..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6UpsertTableSinkFactoryTest.java +++ /dev/null @@ -1,238 +0,0 @@ -/* - * 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.connectors.elasticsearch6; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.formats.json.JsonRowSerializationSchema; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.ElasticsearchUpsertSinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; -import org.apache.flink.streaming.connectors.elasticsearch6.Elasticsearch6UpsertTableSink.DefaultRestClientFactory; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; - -import org.apache.http.HttpHost; -import org.elasticsearch.common.xcontent.XContentType; -import org.junit.Test; - -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_6; -import static org.junit.Assert.assertEquals; - -/** - * Test for {@link Elasticsearch6UpsertTableSink} created by {@link - * Elasticsearch6UpsertTableSinkFactory}. - */ -public class Elasticsearch6UpsertTableSinkFactoryTest - extends ElasticsearchUpsertTableSinkFactoryTestBase { - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - final IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator(INDEX, schema); - - final TestElasticsearch6UpsertTableSink testSink = - new TestElasticsearch6UpsertTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions()); - - final DataStreamMock dataStreamMock = - new DataStreamMock( - new StreamExecutionEnvironmentMock(), - Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - - testSink.consumeDataStream(dataStreamMock); - - final ElasticsearchSink.Builder> expectedBuilder = - new ElasticsearchSink.Builder<>( - Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), - new ElasticsearchUpsertSinkFunction( - indexGenerator, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - Elasticsearch6UpsertTableSink.UPDATE_REQUEST_FACTORY, - new int[0])); - expectedBuilder.setFailureHandler(new DummyFailureHandler()); - expectedBuilder.setBulkFlushBackoff(true); - expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - expectedBuilder.setBulkFlushBackoffDelay(123); - expectedBuilder.setBulkFlushBackoffRetries(3); - expectedBuilder.setBulkFlushInterval(100); - expectedBuilder.setBulkFlushMaxActions(1000); - expectedBuilder.setBulkFlushMaxSizeMb(1); - expectedBuilder.setRestClientFactory(new DefaultRestClientFactory(100, "/myapp")); - assertEquals(expectedBuilder, testSink.builder); - } - - @Override - protected String getElasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_6; - } - - @Override - protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator) { - return new Elasticsearch6UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - private static class TestElasticsearch6UpsertTableSink extends Elasticsearch6UpsertTableSink { - - public ElasticsearchSink.Builder> builder; - - public TestElasticsearch6UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { - builder = super.createBuilder(upsertSinkFunction, httpHosts); - return builder; - } - } - - private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - throw new UnsupportedOperationException(); - } - } - - private static class DataStreamMock extends DataStream> { - - public SinkFunction sinkFunction; - - public DataStreamMock( - StreamExecutionEnvironment environment, - TypeInformation> outType) { - super(environment, new TransformationMock("name", outType, 1)); - } - - @Override - public DataStreamSink> addSink( - SinkFunction> sinkFunction) { - this.sinkFunction = sinkFunction; - return super.addSink(sinkFunction); - } - } - - private static class TransformationMock extends Transformation> { - - public TransformationMock( - String name, TypeInformation> outputType, int parallelism) { - super(name, outputType, parallelism); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java deleted file mode 100644 index 06c2d49f..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSink.java +++ /dev/null @@ -1,280 +0,0 @@ -/* - * 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.connectors.elasticsearch7; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.common.xcontent.XContentType; - -import javax.annotation.Nullable; - -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_DELAY; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_ENABLED; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_RETRIES; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_BACKOFF_TYPE; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_INTERVAL; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_ACTIONS; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.BULK_FLUSH_MAX_SIZE; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.DISABLE_FLUSH_ON_CHECKPOINT; -import static org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption.REST_PATH_PREFIX; - -/** Version-specific upsert table sink for Elasticsearch 7. */ -@Internal -public class Elasticsearch7UpsertTableSink extends ElasticsearchUpsertTableSinkBase { - - @VisibleForTesting - static final RequestFactory UPDATE_REQUEST_FACTORY = new Elasticsearch7RequestFactory(); - - public Elasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - "", - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @VisibleForTesting - Elasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions, - UPDATE_REQUEST_FACTORY); - } - - @Override - protected ElasticsearchUpsertTableSinkBase copy( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - RequestFactory requestFactory) { - - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected SinkFunction> createSinkFunction( - List hosts, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - ElasticsearchUpsertSinkFunction upsertSinkFunction) { - - final List httpHosts = - hosts.stream() - .map((host) -> new HttpHost(host.hostname, host.port, host.protocol)) - .collect(Collectors.toList()); - - final ElasticsearchSink.Builder> builder = - createBuilder(upsertSinkFunction, httpHosts); - - builder.setFailureHandler(failureHandler); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_ACTIONS)) - .ifPresent(v -> builder.setBulkFlushMaxActions(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_MAX_SIZE)) - .ifPresent(v -> builder.setBulkFlushMaxSizeMb(MemorySize.parse(v).getMebiBytes())); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_INTERVAL)) - .ifPresent(v -> builder.setBulkFlushInterval(Long.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_ENABLED)) - .ifPresent(v -> builder.setBulkFlushBackoff(Boolean.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_TYPE)) - .ifPresent( - v -> - builder.setBulkFlushBackoffType( - ElasticsearchSinkBase.FlushBackoffType.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_RETRIES)) - .ifPresent(v -> builder.setBulkFlushBackoffRetries(Integer.valueOf(v))); - - Optional.ofNullable(sinkOptions.get(BULK_FLUSH_BACKOFF_DELAY)) - .ifPresent(v -> builder.setBulkFlushBackoffDelay(Long.valueOf(v))); - - builder.setRestClientFactory( - new DefaultRestClientFactory(sinkOptions.get(REST_PATH_PREFIX))); - - final ElasticsearchSink> sink = builder.build(); - - Optional.ofNullable(sinkOptions.get(DISABLE_FLUSH_ON_CHECKPOINT)) - .ifPresent( - v -> { - if (Boolean.valueOf(v)) { - sink.disableFlushOnCheckpoint(); - } - }); - - return sink; - } - - @VisibleForTesting - ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { - return new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - /** Serializable {@link RestClientFactory} used by the sink. */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the - * sink. - */ - private static class Elasticsearch7RequestFactory implements RequestFactory { - - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, String docType, XContentType contentType, byte[] document) { - return new IndexRequest(index).source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, key); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java deleted file mode 100644 index 6cded3c2..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactory.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.connectors.elasticsearch7; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryBase; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.sinks.UpsertStreamTableSink; -import org.apache.flink.types.Row; - -import org.elasticsearch.common.xcontent.XContentType; - -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; - -/** Table factory for creating an {@link UpsertStreamTableSink} for Elasticsearch 7. */ -@Internal -public class Elasticsearch7UpsertTableSinkFactory extends ElasticsearchUpsertTableSinkFactoryBase { - - @Override - protected String elasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_7; - } - - @Override - protected ElasticsearchUpsertTableSinkBase createElasticsearchUpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 5bcb91f3..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +0,0 @@ -# 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. - -org.apache.flink.streaming.connectors.elasticsearch7.Elasticsearch7UpsertTableSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java deleted file mode 100644 index 7b9f4801..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7UpsertTableSinkFactoryTest.java +++ /dev/null @@ -1,237 +0,0 @@ -/* - * 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.connectors.elasticsearch7; - -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.formats.json.JsonRowSerializationSchema; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.graph.StreamGraph; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.ElasticsearchUpsertSinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.Host; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkBase.SinkOption; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchUpsertTableSinkFactoryTestBase; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGenerator; -import org.apache.flink.streaming.connectors.elasticsearch.index.IndexGeneratorFactory; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; - -import org.apache.http.HttpHost; -import org.elasticsearch.common.xcontent.XContentType; -import org.junit.Test; - -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import static org.apache.flink.table.descriptors.ElasticsearchValidator.CONNECTOR_VERSION_VALUE_7; -import static org.junit.Assert.assertEquals; - -/** - * Test for {@link Elasticsearch7UpsertTableSink} created by {@link - * Elasticsearch7UpsertTableSinkFactory}. - */ -public class Elasticsearch7UpsertTableSinkFactoryTest - extends ElasticsearchUpsertTableSinkFactoryTestBase { - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - final IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator(INDEX, schema); - - final TestElasticsearch7UpsertTableSink testSink = - new TestElasticsearch7UpsertTableSink( - false, - schema, - Collections.singletonList(new Host(HOSTNAME, PORT, SCHEMA)), - INDEX, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - new DummyFailureHandler(), - createTestSinkOptions()); - - final DataStreamMock dataStreamMock = - new DataStreamMock( - new StreamExecutionEnvironmentMock(), - Types.TUPLE(Types.BOOLEAN, schema.toRowType())); - - testSink.consumeDataStream(dataStreamMock); - - final ElasticsearchSink.Builder> expectedBuilder = - new ElasticsearchSink.Builder<>( - Collections.singletonList(new HttpHost(HOSTNAME, PORT, SCHEMA)), - new ElasticsearchUpsertSinkFunction( - indexGenerator, - DOC_TYPE, - KEY_DELIMITER, - KEY_NULL_LITERAL, - JsonRowSerializationSchema.builder() - .withTypeInfo(schema.toRowType()) - .build(), - XContentType.JSON, - Elasticsearch7UpsertTableSink.UPDATE_REQUEST_FACTORY, - new int[0])); - expectedBuilder.setFailureHandler(new DummyFailureHandler()); - expectedBuilder.setBulkFlushBackoff(true); - expectedBuilder.setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - expectedBuilder.setBulkFlushBackoffDelay(123); - expectedBuilder.setBulkFlushBackoffRetries(3); - expectedBuilder.setBulkFlushInterval(100); - expectedBuilder.setBulkFlushMaxActions(1000); - expectedBuilder.setBulkFlushMaxSizeMb(1); - expectedBuilder.setRestClientFactory( - new Elasticsearch7UpsertTableSink.DefaultRestClientFactory("/myapp")); - assertEquals(expectedBuilder, testSink.builder); - } - - @Override - protected String getElasticsearchVersion() { - return CONNECTOR_VERSION_VALUE_7; - } - - @Override - protected ElasticsearchUpsertTableSinkBase getExpectedTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions, - IndexGenerator indexGenerator) { - return new Elasticsearch7UpsertTableSink( - isAppendOnly, - schema, - hosts, - index, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - // -------------------------------------------------------------------------------------------- - // Helper classes - // -------------------------------------------------------------------------------------------- - - private static class TestElasticsearch7UpsertTableSink extends Elasticsearch7UpsertTableSink { - - public ElasticsearchSink.Builder> builder; - - public TestElasticsearch7UpsertTableSink( - boolean isAppendOnly, - TableSchema schema, - List hosts, - String index, - String docType, - String keyDelimiter, - String keyNullLiteral, - SerializationSchema serializationSchema, - XContentType contentType, - ActionRequestFailureHandler failureHandler, - Map sinkOptions) { - - super( - isAppendOnly, - schema, - hosts, - index, - docType, - keyDelimiter, - keyNullLiteral, - serializationSchema, - contentType, - failureHandler, - sinkOptions); - } - - @Override - protected ElasticsearchSink.Builder> createBuilder( - ElasticsearchUpsertSinkFunction upsertSinkFunction, List httpHosts) { - builder = super.createBuilder(upsertSinkFunction, httpHosts); - return builder; - } - } - - private static class StreamExecutionEnvironmentMock extends StreamExecutionEnvironment { - - @Override - public JobExecutionResult execute(StreamGraph streamGraph) throws Exception { - throw new UnsupportedOperationException(); - } - } - - private static class DataStreamMock extends DataStream> { - - public SinkFunction sinkFunction; - - public DataStreamMock( - StreamExecutionEnvironment environment, - TypeInformation> outType) { - super(environment, new TransformationMock("name", outType, 1)); - } - - @Override - public DataStreamSink> addSink( - SinkFunction> sinkFunction) { - this.sinkFunction = sinkFunction; - return super.addSink(sinkFunction); - } - } - - private static class TransformationMock extends Transformation> { - - public TransformationMock( - String name, TypeInformation> outputType, int parallelism) { - super(name, outputType, parallelism); - } - - @Override - public List> getTransitivePredecessors() { - return null; - } - - @Override - public List> getInputs() { - return Collections.emptyList(); - } - } -} From c4af18eb8c1e541e2c9f12dbb0e8255f8af8c45b Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Mon, 9 Aug 2021 11:51:52 +0800 Subject: [PATCH 145/207] [hotfix][test/testcontainers] Use Testcontainer BOM to manage versions in the project --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 1 - flink-connectors/flink-connector-elasticsearch7/pom.xml | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 6491887f..e62bd26f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -91,7 +91,6 @@ under the License. org.testcontainers elasticsearch - 1.15.1 test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 0141d697..392c9a8d 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -89,7 +89,6 @@ under the License. org.testcontainers elasticsearch - 1.15.1 test From 0f508ca4fed5f5c36ab4d694e021b2e3a89e92ff Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 21 Aug 2021 13:37:57 +0200 Subject: [PATCH 146/207] [hotfix][tests] Let ElasticSearch6DynamicSink tests extend TestLogger --- .../table/Elasticsearch6DynamicSinkFactoryTest.java | 3 ++- .../elasticsearch/table/Elasticsearch6DynamicSinkITCase.java | 3 ++- .../elasticsearch/table/Elasticsearch6DynamicSinkTest.java | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 25bcbbfb..e99abbe9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; @@ -35,7 +36,7 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ -public class Elasticsearch6DynamicSinkFactoryTest { +public class Elasticsearch6DynamicSinkFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); @Test diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index b7090c51..908555e5 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; @@ -65,7 +66,7 @@ import static org.junit.Assert.assertThat; /** IT tests for {@link Elasticsearch6DynamicSink}. */ -public class Elasticsearch6DynamicSinkITCase { +public class Elasticsearch6DynamicSinkITCase extends TestLogger { @ClassRule public static ElasticsearchContainer elasticsearchContainer = diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index f654222e..6ad4f78b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; +import org.apache.flink.util.TestLogger; import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; @@ -46,7 +47,7 @@ import static org.mockito.Mockito.verify; /** Tests for {@link Elasticsearch6DynamicSink} parameters. */ -public class Elasticsearch6DynamicSinkTest { +public class Elasticsearch6DynamicSinkTest extends TestLogger { private static final String FIELD_KEY = "key"; private static final String FIELD_FRUIT_NAME = "fruit_name"; From f3fc49736d59a2a0da5ed846c4f8ee92d854eb65 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 21 Aug 2021 13:53:33 +0200 Subject: [PATCH 147/207] [FLINK-21538][tests] Set default parallelism to 4 for Elasticsearch6DynamicSinkITCase.testWritingDocuments This commit sets the default parallelism of Elasticsearch6DynamicSinkITCase.testWritingDocuments to 4 in order to reduce the load for our CI infrastructure. This closes #16918. --- .../elasticsearch/table/Elasticsearch6DynamicSinkITCase.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 908555e5..77449d59 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -140,6 +140,8 @@ public void testWritingDocuments() throws Exception { SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + rowData.setRowKind(RowKind.UPDATE_AFTER); environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); From 401629b45ec6f2f4bd11a8b9b6f1e60026d1ad14 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 21 Aug 2021 14:20:19 +0200 Subject: [PATCH 148/207] [hotfix] Let Elasticsearch7DynamicSink* test cases extend from TestLogger --- .../table/Elasticsearch7DynamicSinkFactoryTest.java | 3 ++- .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 3 ++- .../elasticsearch/table/Elasticsearch7DynamicSinkTest.java | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index ba37dbc5..3ff21ed1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; @@ -35,7 +36,7 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ -public class Elasticsearch7DynamicSinkFactoryTest { +public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { @Rule public ExpectedException thrown = ExpectedException.none(); @Test diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 18c0b3e1..2bf6c365 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -37,6 +37,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.client.Client; @@ -65,7 +66,7 @@ import static org.junit.Assert.assertThat; /** IT tests for {@link Elasticsearch7DynamicSink}. */ -public class Elasticsearch7DynamicSinkITCase { +public class Elasticsearch7DynamicSinkITCase extends TestLogger { @ClassRule public static ElasticsearchContainer elasticsearchContainer = diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index 54c75741..e8854b22 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; +import org.apache.flink.util.TestLogger; import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; @@ -46,7 +47,7 @@ import static org.mockito.Mockito.verify; /** Tests for {@link Elasticsearch7DynamicSink} parameters. */ -public class Elasticsearch7DynamicSinkTest { +public class Elasticsearch7DynamicSinkTest extends TestLogger { private static final String FIELD_KEY = "key"; private static final String FIELD_FRUIT_NAME = "fruit_name"; From 81ee52c3923fc5c8338b956be6498a5a17bf9369 Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sat, 21 Aug 2021 14:21:33 +0200 Subject: [PATCH 149/207] [FLINK-22333][tests] Harden Elasticsearch7DynamicSinkITCase.testWritingDocuments by setting parallelism to 4 This commit hardens the Elasticsearch7DynamicSinkITCase.testWritingDocuments tests by settings its parallelism to 4. Otherwise the test is run with as many CPUs are available on the machine. This can slow down the test on our CI infrastructure. This closes #16924. --- .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 2bf6c365..a74496ca 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -135,6 +135,8 @@ public void testWritingDocuments() throws Exception { SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + rowData.setRowKind(RowKind.UPDATE_AFTER); environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); From fa9d345a157b5bc612eb12e42a87369e67feca2e Mon Sep 17 00:00:00 2001 From: Xintong Song Date: Sat, 28 Aug 2021 08:33:55 +0800 Subject: [PATCH 150/207] Update version to 1.15-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 07817acc..5a94cb9a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index e62bd26f..d2a56ae8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 392c9a8d..90b5da2e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index e83cfe2e..b0a09873 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 255af93f..6fe470b9 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.14-SNAPSHOT + 1.15-SNAPSHOT .. From 21d21b76eae00aebfccfb8597221df8c618e969d Mon Sep 17 00:00:00 2001 From: MartijnVisser Date: Wed, 8 Sep 2021 13:23:09 +0200 Subject: [PATCH 151/207] [FLINK-24034] Upgrade apache commons dependencies --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 4bc2241c..e28bbc8c 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -12,7 +12,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 -- commons-codec:commons-codec:1.13 +- commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 - org.apache.httpcomponents:httpclient:4.5.13 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index d14c8181..4617b41b 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -14,7 +14,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 - com.github.spullara.mustache.java:compiler:0.9.6 -- commons-codec:commons-codec:1.13 +- commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.4 - org.apache.httpcomponents:httpclient:4.5.13 From 0bab586ad7b0ac36df2d4edf8f7f081c1b66d665 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Wed, 25 Aug 2021 11:13:26 +0200 Subject: [PATCH 152/207] [FLINK-23607][state/changelog] Make Changelog Backend a transitive dependency ...of flink-test-utils instead of direct dependency of each module. --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 7 ------- flink-connectors/flink-connector-elasticsearch7/pom.xml | 7 ------- 2 files changed, 14 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index d2a56ae8..6790c36d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -109,13 +109,6 @@ under the License. test-jar - - org.apache.flink - flink-statebackend-changelog - ${project.version} - test - - org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 90b5da2e..de02b967 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -107,13 +107,6 @@ under the License. test-jar - - org.apache.flink - flink-statebackend-changelog - ${project.version} - test - - org.apache.flink flink-connector-elasticsearch-base_${scala.binary.version} From 050d4d39d0a6492540e3f9145673d386b2a5a4c6 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Wed, 29 Sep 2021 17:48:02 +0200 Subject: [PATCH 153/207] [FLINK-24399][table-common] Add DynamicTableSource.Context#createTypeInformation(LogicalType) and DynamicTableSink.Context#createTypeInformation(LogicalType) Signed-off-by: slinkydeveloper --- .../table/Elasticsearch6DynamicSinkITCase.java | 6 ++++++ .../elasticsearch/table/Elasticsearch6DynamicSinkTest.java | 6 ++++++ .../table/Elasticsearch7DynamicSinkITCase.java | 6 ++++++ .../elasticsearch/table/Elasticsearch7DynamicSinkTest.java | 6 ++++++ 4 files changed, 24 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 77449d59..0ebc52b5 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -35,6 +35,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; @@ -384,6 +385,11 @@ public TypeInformation createTypeInformation(DataType consumedDataType) { return null; } + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + @Override public DynamicTableSink.DataStructureConverter createDataStructureConverter( DataType consumedDataType) { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java index 6ad4f78b..d1af3ec7 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.util.TestLogger; import org.apache.http.HttpHost; @@ -256,6 +257,11 @@ public TypeInformation createTypeInformation(DataType consumedDataType) { return null; } + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + @Override public DynamicTableSink.DataStructureConverter createDataStructureConverter( DataType consumedDataType) { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index a74496ca..fdf7a3a0 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -35,6 +35,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; @@ -361,6 +362,11 @@ public TypeInformation createTypeInformation(DataType consumedDataType) { return null; } + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + @Override public DynamicTableSink.DataStructureConverter createDataStructureConverter( DataType consumedDataType) { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java index e8854b22..6ab6828f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -33,6 +33,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.util.TestLogger; import org.apache.http.HttpHost; @@ -256,6 +257,11 @@ public TypeInformation createTypeInformation(DataType consumedDataType) { return null; } + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + @Override public DynamicTableSink.DataStructureConverter createDataStructureConverter( DataType consumedDataType) { From 6f47268cf546076b020da069b831b88299d0b1c9 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Mon, 20 Sep 2021 16:59:26 +0200 Subject: [PATCH 154/207] [FLINK-24324][connectors/elasticsearch] Add Elasticsearch 7 sink based on the unified sink (FLIP-143) --- .../sink/BulkProcessorConfig.java | 71 ++++ .../sink/ElasticsearchEmitter.java | 77 ++++ .../elasticsearch/sink/ElasticsearchSink.java | 126 +++++++ .../sink/ElasticsearchSinkBuilder.java | 246 +++++++++++++ .../sink/ElasticsearchWriter.java | 332 ++++++++++++++++++ .../elasticsearch/sink/FlushBackoffType.java | 37 ++ .../sink/NetworkClientConfig.java | 53 +++ .../elasticsearch/sink/RequestIndexer.java | 55 +++ .../sink/ElasticsearchSinkBuilderTest.java | 85 +++++ .../sink/ElasticsearchSinkITCase.java | 229 ++++++++++++ .../sink/ElasticsearchWriterITCase.java | 272 ++++++++++++++ .../elasticsearch/sink/TestClient.java | 77 ++++ .../elasticsearch/sink/TestEmitter.java | 75 ++++ 13 files changed, 1735 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java new file mode 100644 index 00000000..8ec88b6c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java @@ -0,0 +1,71 @@ +/* + * 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.connector.elasticsearch.sink; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +class BulkProcessorConfig implements Serializable { + + private final int bulkFlushMaxActions; + private final int bulkFlushMaxMb; + private final long bulkFlushInterval; + private final FlushBackoffType flushBackoffType; + private final int bulkFlushBackoffRetries; + private final long bulkFlushBackOffDelay; + + BulkProcessorConfig( + int bulkFlushMaxActions, + int bulkFlushMaxMb, + long bulkFlushInterval, + FlushBackoffType flushBackoffType, + int bulkFlushBackoffRetries, + long bulkFlushBackOffDelay) { + this.bulkFlushMaxActions = bulkFlushMaxActions; + this.bulkFlushMaxMb = bulkFlushMaxMb; + this.bulkFlushInterval = bulkFlushInterval; + this.flushBackoffType = checkNotNull(flushBackoffType); + this.bulkFlushBackoffRetries = bulkFlushBackoffRetries; + this.bulkFlushBackOffDelay = bulkFlushBackOffDelay; + } + + public int getBulkFlushMaxActions() { + return bulkFlushMaxActions; + } + + public int getBulkFlushMaxMb() { + return bulkFlushMaxMb; + } + + public long getBulkFlushInterval() { + return bulkFlushInterval; + } + + public FlushBackoffType getFlushBackoffType() { + return flushBackoffType; + } + + public int getBulkFlushBackoffRetries() { + return bulkFlushBackoffRetries; + } + + public long getBulkFlushBackOffDelay() { + return bulkFlushBackOffDelay; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java new file mode 100644 index 00000000..ad368ad5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java @@ -0,0 +1,77 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.connector.sink.SinkWriter; + +import org.elasticsearch.action.ActionRequest; + +/** + * Creates none or multiple {@link ActionRequest ActionRequests} from the incoming elements. + * + *

This is used by sinks to prepare elements for sending them to Elasticsearch. + * + *

Example: + * + *

{@code
+ * private static class TestElasticsearchEmitter implements ElasticsearchEmitter> {
+ *
+ *     public IndexRequest createIndexRequest(Tuple2 element) {
+ *         Map document = new HashMap<>();
+ * 		   document.put("data", element.f1);
+ *
+ * 	       return Requests.indexRequest()
+ * 		       .index("my-index")
+ * 			   .type("my-type")
+ * 			   .id(element.f0.toString())
+ * 			   .source(document);
+ *     }
+ *
+ * 	   public void emit(Tuple2 element, RequestIndexer indexer) {
+ * 	       indexer.add(createIndexRequest(element));
+ *     }
+ * }
+ *
+ * }
+ * + * @param The type of the element handled by this {@link ElasticsearchEmitter} + */ +@PublicEvolving +public interface ElasticsearchEmitter extends Function { + + /** + * Initialization method for the function. It is called once before the actual working process + * methods. + */ + default void open() throws Exception {} + + /** Tear-down method for the function. It is called when the sink closes. */ + default void close() throws Exception {} + + /** + * Process the incoming element to produce multiple {@link ActionRequest ActionRequests}. The + * produced requests should be added to the provided {@link RequestIndexer}. + * + * @param element incoming element to process + * @param context to access additional information about the record + * @param indexer request indexer that {@code ActionRequest} should be added to + */ + void emit(T element, SinkWriter.Context context, RequestIndexer indexer); +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java new file mode 100644 index 00000000..37151f79 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -0,0 +1,126 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink.Committer; +import org.apache.flink.api.connector.sink.GlobalCommitter; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import org.apache.http.HttpHost; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Flink Sink to insert or update data in an Elasticsearch index. The sink supports the following + * delivery guarantees. + * + *
    + *
  • {@link DeliveryGuarantee#NONE} does not provide any guarantees: actions are flushed to + * Elasticsearch only depending on the configurations of the bulk processor. In case of a + * failure, it might happen that actions are lost if the bulk processor still has buffered + * actions. + *
  • {@link DeliveryGuarantee#AT_LEAST_ONCE} on a checkpoint the sink will wait until all + * buffered actions are flushed to and acknowledged by Elasticsearch. No actions will be lost + * but actions might be sent to Elasticsearch multiple times when Flink restarts. These + * additional requests may cause inconsistent data in ElasticSearch right after the restart, + * but eventually everything will be consistent again. + *
+ * + * @param type of the records converted to Elasticsearch actions + * @see ElasticsearchSinkBuilder on how to construct a ElasticsearchSink + */ +@PublicEvolving +public class ElasticsearchSink implements Sink { + + private final List hosts; + private final ElasticsearchEmitter emitter; + private final BulkProcessorConfig buildBulkProcessorConfig; + private final NetworkClientConfig networkClientConfig; + private final DeliveryGuarantee deliveryGuarantee; + + ElasticsearchSink( + List hosts, + ElasticsearchEmitter emitter, + DeliveryGuarantee deliveryGuarantee, + BulkProcessorConfig buildBulkProcessorConfig, + NetworkClientConfig networkClientConfig) { + this.hosts = checkNotNull(hosts); + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + this.emitter = checkNotNull(emitter); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.buildBulkProcessorConfig = checkNotNull(buildBulkProcessorConfig); + this.networkClientConfig = checkNotNull(networkClientConfig); + } + + /** + * Create a {@link ElasticsearchSinkBuilder} to construct a new {@link ElasticsearchSink}. + * + * @param type of incoming records + * @return {@link ElasticsearchSinkBuilder} + */ + public static ElasticsearchSinkBuilder builder() { + return new ElasticsearchSinkBuilder<>(); + } + + @Override + public SinkWriter createWriter(InitContext context, List states) + throws IOException { + return new ElasticsearchWriter<>( + hosts, + emitter, + deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE, + buildBulkProcessorConfig, + networkClientConfig, + context.metricGroup(), + context.getMailboxExecutor()); + } + + @Override + public Optional> getWriterStateSerializer() { + return Optional.empty(); + } + + @Override + public Optional> createCommitter() throws IOException { + return Optional.empty(); + } + + @Override + public Optional> createGlobalCommitter() throws IOException { + return Optional.empty(); + } + + @Override + public Optional> getCommittableSerializer() { + return Optional.empty(); + } + + @Override + public Optional> getGlobalCommittableSerializer() { + return Optional.empty(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java new file mode 100644 index 00000000..68e30cb3 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java @@ -0,0 +1,246 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Builder to construct a {@link ElasticsearchSink}. + * + *

The following example shows the minimal setup to create a ElasticsearchSink that submits + * actions on checkpoint or the default number of actions was buffered (1000). + * + *

{@code
+ * Elasticsearch sink = Elasticsearch
+ *     .builder()
+ *     .setHosts(MY_ELASTICSEARCH_HOSTS)
+ *     .setEmitter(MY_ELASTICSEARCH_EMITTER)
+ *     .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+ *     .build();
+ * }
+ * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class ElasticsearchSinkBuilder { + + private int bulkFlushMaxActions = 1000; + private int bulkFlushMaxMb = -1; + private long bulkFlushInterval = -1; + private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE; + private int bulkFlushBackoffRetries = -1; + private long bulkFlushBackOffDelay = -1; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private List hosts; + private ElasticsearchEmitter emitter; + private String username; + private String password; + private String connectionPathPrefix; + + ElasticsearchSinkBuilder() {} + + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + */ + public ElasticsearchSinkBuilder setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return this; + } + + /** + * Sets the emitter which is invoked on every record to convert it to Elasticsearch actions. + * + * @param emitter to process records into Elasticsearch actions. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setEmitter( + ElasticsearchEmitter emitter) { + checkNotNull(emitter); + checkState( + InstantiationUtil.isSerializable(emitter), + "The elasticsearch emitter must be serializable."); + final ElasticsearchSinkBuilder self = self(); + self.emitter = emitter; + return self; + } + + /** + * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link + * DeliveryGuarantee#NONE} + * + * @param deliveryGuarantee which describes the record emission behaviour + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { + checkState( + deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, + "Elasticsearch sink does not support the EXACTLY_ONCE guarantee."); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + return this; + } + + /** + * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to + * disable it. The default flush size 1000. + * + * @param numMaxActions the maximum number of actions to buffer per bulk request. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setBulkFlushMaxActions(int numMaxActions) { + checkState( + numMaxActions == -1 || numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkFlushMaxActions = numMaxActions; + return this; + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. You can pass -1 to + * disable it. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setBulkFlushMaxSizeMb(int maxSizeMb) { + checkState( + maxSizeMb == -1 || maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkFlushMaxMb = maxSizeMb; + return this; + } + + /** + * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setBulkFlushInterval(long intervalMillis) { + checkState( + intervalMillis == -1 || intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than " + + "or equal to 0."); + this.bulkFlushInterval = intervalMillis; + return this; + } + + /** + * Sets the type of back of to use when flushing bulk requests. The default bulk flush back off + * type is {@link FlushBackoffType#NONE}. + * + *

Sets the amount of delay between each backoff attempt when flushing bulk requests, in + * milliseconds. + * + *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setBulkFlushBackoffStrategy( + FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) { + this.bulkFlushBackoffType = checkNotNull(flushBackoffType); + checkState( + flushBackoffType != FlushBackoffType.NONE, + "FlushBackoffType#NONE does not require a configuration it is the default, retries and delay are ignored."); + checkState(maxRetries > 0, "Max number of backoff attempts must be larger than 0."); + this.bulkFlushBackoffRetries = maxRetries; + checkState( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger " + + "than or equal to 0."); + this.bulkFlushBackOffDelay = delayMillis; + return this; + } + + /** + * Sets the username used to authenticate the connection with the Elasticsearch cluster. + * + * @param username of the Elasticsearch cluster user + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setConnectionUsername(String username) { + checkNotNull(username); + this.username = username; + return this; + } + + /** + * Sets the password used to authenticate the conection with the Elasticsearch cluster. + * + * @param password of the Elasticsearch cluster user + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setConnectionPassword(String password) { + checkNotNull(password); + this.password = password; + return this; + } + + /** + * Sets a prefix which used for every REST communication to the Elasticsearch cluster. + * + * @param prefix for the communication + * @return {@link ElasticsearchSinkBuilder} + */ + public ElasticsearchSinkBuilder setConnectionPathPrefix(String prefix) { + checkNotNull(prefix); + this.connectionPathPrefix = prefix; + return this; + } + + /** @return {@link ElasticsearchSink} */ + public ElasticsearchSink build() { + checkNotNull(hosts); + checkNotNull(emitter); + return new ElasticsearchSink( + hosts, + emitter, + deliveryGuarantee, + buildBulkProcessorConfig(), + new NetworkClientConfig(username, password, connectionPathPrefix)); + } + + @SuppressWarnings("unchecked") + private ElasticsearchSinkBuilder self() { + return (ElasticsearchSinkBuilder) this; + } + + private BulkProcessorConfig buildBulkProcessorConfig() { + return new BulkProcessorConfig( + bulkFlushMaxActions, + bulkFlushMaxMb, + bulkFlushInterval, + bulkFlushBackoffType, + bulkFlushBackoffRetries, + bulkFlushBackOffDelay); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java new file mode 100644 index 00000000..b024bb24 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -0,0 +1,332 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.rest.RestStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed; +import static org.apache.flink.util.Preconditions.checkNotNull; + +class ElasticsearchWriter implements SinkWriter { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); + + private final ElasticsearchEmitter emitter; + private final MailboxExecutor mailboxExecutor; + private final boolean flushOnCheckpoint; + private final BulkProcessor bulkProcessor; + private final RestHighLevelClient client; + private final RequestIndexer requestIndexer; + private final Counter numBytesOutCounter; + + private long pendingActions = 0; + private boolean checkpointInProgress = false; + private volatile long lastSendTime = 0; + private volatile long ackTime = Long.MAX_VALUE; + private volatile boolean closed = false; + + /** + * Constructor creating an elasticsearch writer. + * + * @param hosts the reachable elasticsearch cluster nodes + * @param emitter converting incoming records to elasticsearch actions + * @param flushOnCheckpoint if true all until now received records are flushed after every + * checkpoint + * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link + * BulkProcessor} + * @param networkClientConfig describing properties of the network connection used to connect to + * the elasticsearch cluster + * @param metricGroup for the sink writer + * @param mailboxExecutor Flink's mailbox executor + */ + public ElasticsearchWriter( + List hosts, + ElasticsearchEmitter emitter, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + NetworkClientConfig networkClientConfig, + SinkWriterMetricGroup metricGroup, + MailboxExecutor mailboxExecutor) { + this.emitter = checkNotNull(emitter); + this.flushOnCheckpoint = flushOnCheckpoint; + this.mailboxExecutor = checkNotNull(mailboxExecutor); + this.client = + new RestHighLevelClient( + configureRestClientBuilder( + RestClient.builder(hosts.toArray(new HttpHost[0])), + networkClientConfig)); + this.bulkProcessor = + configureBulkProcessor( + BulkProcessor.builder( + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener), + new BulkListener()), + bulkProcessorConfig); + this.requestIndexer = new DefaultRequestIndexer(); + checkNotNull(metricGroup); + metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); + this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); + try { + emitter.open(); + } catch (Exception e) { + throw new FlinkRuntimeException("Failed to open the ElasticsearchEmitter", e); + } + } + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // do not allow new bulk writes until all actions are flushed + while (checkpointInProgress) { + mailboxExecutor.yield(); + } + emitter.emit(element, context, requestIndexer); + } + + @Override + public List prepareCommit(boolean flush) throws IOException, InterruptedException { + checkpointInProgress = true; + while (pendingActions != 0 && (flushOnCheckpoint || flush)) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + checkpointInProgress = false; + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + closed = true; + emitter.close(); + client.close(); + } + + private static RestClientBuilder configureRestClientBuilder( + RestClientBuilder builder, NetworkClientConfig networkClientConfig) { + if (networkClientConfig.getConnectionPathPrefix() != null) { + builder.setPathPrefix(networkClientConfig.getConnectionPathPrefix()); + } + if (networkClientConfig.getPassword() != null + && networkClientConfig.getUsername() != null) { + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials( + networkClientConfig.getUsername(), networkClientConfig.getPassword())); + builder.setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); + } + return builder; + } + + private static BulkProcessor configureBulkProcessor( + BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { + // This makes flush() blocking + builder.setConcurrentRequests(0); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + + return builder.build(); + } + + private class BulkListener implements BulkProcessor.Listener { + + @Override + public void beforeBulk(long executionId, BulkRequest request) { + LOG.info("Sending bulk of {} actions to Elasticsearch.", request.numberOfActions()); + lastSendTime = System.currentTimeMillis(); + numBytesOutCounter.inc(request.estimatedSizeInBytes()); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + ackTime = System.currentTimeMillis(); + enqueueActionInMailbox( + () -> extractFailures(request, response), "elasticsearchSuccessCallback"); + } + + @Override + public void afterBulk(long executionId, BulkRequest request, Throwable failure) { + enqueueActionInMailbox( + () -> { + throw new FlinkRuntimeException("Complete bulk has failed.", failure); + }, + "elasticsearchErrorCallback"); + } + } + + private void enqueueActionInMailbox( + ThrowingRunnable action, String actionName) { + // If the writer is cancelled before the last bulk response (i.e. no flush on checkpoint + // configured or shutdown without a final + // checkpoint) the mailbox might already be shutdown, so we should not enqueue any + // actions. + if (isClosed()) { + return; + } + mailboxExecutor.execute(action, actionName); + } + + private void extractFailures(BulkRequest request, BulkResponse response) { + if (!response.hasFailures()) { + pendingActions -= request.numberOfActions(); + return; + } + + Throwable chainedFailures = null; + for (int i = 0; i < response.getItems().length; i++) { + final BulkItemResponse itemResponse = response.getItems()[i]; + if (!itemResponse.isFailed()) { + continue; + } + final Throwable failure = itemResponse.getFailure().getCause(); + if (failure == null) { + continue; + } + final RestStatus restStatus = itemResponse.getFailure().getStatus(); + final DocWriteRequest actionRequest = request.requests().get(i); + + chainedFailures = + firstOrSuppressed( + wrapException(restStatus, failure, actionRequest), chainedFailures); + } + if (chainedFailures == null) { + return; + } + throw new FlinkRuntimeException(chainedFailures); + } + + private static Throwable wrapException( + RestStatus restStatus, Throwable rootFailure, DocWriteRequest actionRequest) { + if (restStatus == null) { + return new FlinkRuntimeException( + String.format("Single action %s of bulk request failed.", actionRequest), + rootFailure); + } else { + return new FlinkRuntimeException( + String.format( + "Single action %s of bulk request failed with status %s.", + actionRequest, restStatus.getStatus()), + rootFailure); + } + } + + private boolean isClosed() { + if (closed) { + LOG.warn("Writer was closed before all records were acknowledged by Elasticsearch."); + } + return closed; + } + + private class DefaultRequestIndexer implements RequestIndexer { + + @Override + public void add(DeleteRequest... deleteRequests) { + for (final DeleteRequest deleteRequest : deleteRequests) { + pendingActions++; + bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (final IndexRequest indexRequest : indexRequests) { + pendingActions++; + bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (final UpdateRequest updateRequest : updateRequests) { + pendingActions++; + bulkProcessor.add(updateRequest); + } + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java new file mode 100644 index 00000000..5a33208f --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java @@ -0,0 +1,37 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Used to control whether the sink should retry failed requests at all or with which kind back off + * strategy. + */ +@PublicEvolving +public enum FlushBackoffType { + /** After every failure, it waits a configured time until the retries are exhausted. */ + CONSTANT, + /** + * After every failure, it waits initially the configured time and increases the waiting time + * exponentially until the retries are exhausted. + */ + EXPONENTIAL, + /** The failure is not retried. */ + NONE, +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java new file mode 100644 index 00000000..647d0213 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java @@ -0,0 +1,53 @@ +/* + * 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.connector.elasticsearch.sink; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +class NetworkClientConfig implements Serializable { + + @Nullable private final String username; + @Nullable private final String password; + @Nullable private final String connectionPathPrefix; + + NetworkClientConfig( + @Nullable String username, + @Nullable String password, + @Nullable String connectionPathPrefix) { + this.username = username; + this.password = password; + this.connectionPathPrefix = connectionPathPrefix; + } + + @Nullable + public String getUsername() { + return username; + } + + @Nullable + public String getPassword() { + return password; + } + + @Nullable + public String getConnectionPathPrefix() { + return connectionPathPrefix; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java new file mode 100644 index 00000000..40669e0c --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java @@ -0,0 +1,55 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; + +/** + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them + * for sending to an Elasticsearch cluster. + */ +@Internal +public interface RequestIndexer { + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to + * Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java new file mode 100644 index 00000000..b72411cb --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.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 org.apache.flink.connector.elasticsearch.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.http.HttpHost; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Tests for {@link ElasticsearchSinkBuilder}. */ +class ElasticsearchSinkBuilderTest extends TestLogger { + + @ParameterizedTest + @MethodSource("validBuilders") + void testBuildElasticsearchSink(ElasticsearchSinkBuilder builder) { + builder.build(); + } + + @Test + void testThrowIfExactlyOnceConfigured() { + assertThrows( + IllegalStateException.class, + () -> createBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); + } + + @Test + void testThrowIfHostsNotSet() { + assertThrows( + NullPointerException.class, + () -> + new ElasticsearchSinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .build()); + } + + @Test + void testThrowIfEmitterNotSet() { + assertThrows( + NullPointerException.class, + () -> + new ElasticsearchSinkBuilder<>() + .setHosts(new HttpHost("localhost:3000")) + .build()); + } + + private static List> validBuilders() { + return Lists.newArrayList( + createBuilder(), + createBuilder().setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), + createBuilder().setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), + createBuilder() + .setConnectionUsername("username") + .setConnectionPassword("password")); + } + + private static ElasticsearchSinkBuilder createBuilder() { + return new ElasticsearchSinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java new file mode 100644 index 00000000..3c0a62a4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java @@ -0,0 +1,229 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.function.BiFunction; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class ElasticsearchSinkITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + private static final String ELASTICSEARCH_PASSWORD = "test-password"; + private static final String ELASTICSEARCH_USER = "elastic"; + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer( + DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_7)) + .withPassword(ELASTICSEARCH_PASSWORD) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + private static boolean failed; + + private RestHighLevelClient client; + private TestClient context; + + @BeforeEach + void setUp() { + failed = false; + final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, + new UsernamePasswordCredentials(ELASTICSEARCH_USER, ELASTICSEARCH_PASSWORD)); + client = + new RestHighLevelClient( + RestClient.builder(HttpHost.create(ES_CONTAINER.getHttpHostAddress())) + .setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider( + credentialsProvider))); + context = new TestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @ParameterizedTest + @EnumSource(DeliveryGuarantee.class) + void testWriteToElasticSearchWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) + throws Exception { + final String index = "test-es-with-delivery-" + deliveryGuarantee; + boolean failure = false; + try { + runTest(index, false, TestEmitter::jsonEmitter, deliveryGuarantee, null); + } catch (IllegalStateException e) { + failure = true; + assertSame(deliveryGuarantee, DeliveryGuarantee.EXACTLY_ONCE); + } finally { + assertEquals(failure, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); + } + } + + @ParameterizedTest + @MethodSource("elasticsearchEmitters") + void testWriteJsonToElasticsearch( + BiFunction>> + emitterProvider) + throws Exception { + final String index = "test-elasticsearch-sink-" + UUID.randomUUID(); + runTest(index, false, emitterProvider, null); + } + + @Test + void testRecovery() throws Exception { + final String index = "test-recovery-elasticsearch-sink"; + runTest(index, true, TestEmitter::jsonEmitter, new FailingMapper()); + assertTrue(failed); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> + emitterProvider, + @Nullable MapFunction additionalMapper) + throws Exception { + runTest( + index, + allowRestarts, + emitterProvider, + DeliveryGuarantee.AT_LEAST_ONCE, + additionalMapper); + } + + private void runTest( + String index, + boolean allowRestarts, + BiFunction>> + emitterProvider, + DeliveryGuarantee deliveryGuarantee, + @Nullable MapFunction additionalMapper) + throws Exception { + final ElasticsearchSink> sink = + ElasticsearchSink.builder() + .setHosts(HttpHost.create(ES_CONTAINER.getHttpHostAddress())) + .setEmitter(emitterProvider.apply(index, context.getDataFieldName())) + .setBulkFlushMaxActions(5) + .setConnectionUsername(ELASTICSEARCH_USER) + .setConnectionPassword(ELASTICSEARCH_PASSWORD) + .setDeliveryGuarantee(deliveryGuarantee) + .build(); + + final Configuration config = new Configuration(); + config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + final StreamExecutionEnvironment env = new LocalStreamEnvironment(config); + env.enableCheckpointing(100L); + if (!allowRestarts) { + env.setRestartStrategy(RestartStrategies.noRestart()); + } + DataStream stream = env.fromSequence(1, 5); + + if (additionalMapper != null) { + stream = stream.map(additionalMapper); + } + + stream.map( + new MapFunction>() { + @Override + public Tuple2 map(Long value) throws Exception { + return Tuple2.of( + value.intValue(), + TestClient.buildMessage(value.intValue())); + } + }) + .sinkTo(sink); + env.execute(); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + } + + private static List>>> + elasticsearchEmitters() { + return Lists.newArrayList(TestEmitter::jsonEmitter, TestEmitter::smileEmitter); + } + + private static class FailingMapper implements MapFunction, CheckpointListener { + + private int emittedRecords = 0; + + @Override + public Long map(Long value) throws Exception { + Thread.sleep(50); + emittedRecords++; + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (failed || emittedRecords == 0) { + return; + } + failed = true; + throw new Exception("Expected failure"); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java new file mode 100644 index 00000000..91779dd1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -0,0 +1,272 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingRunnable; + +import org.apache.http.HttpHost; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.Optional; + +import static org.apache.flink.connector.elasticsearch.sink.TestClient.buildMessage; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThan; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Tests for {@link ElasticsearchWriter}. */ +@Testcontainers +class ElasticsearchWriterITCase extends TestLogger { + + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriterITCase.class); + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + private RestHighLevelClient client; + private TestClient context; + private MetricListener metricListener; + + @BeforeEach + void setUp() { + metricListener = new MetricListener(); + client = + new RestHighLevelClient( + RestClient.builder(HttpHost.create(ES_CONTAINER.getHttpHostAddress()))); + context = new TestClient(client); + } + + @AfterEach + void tearDown() throws IOException { + if (client != null) { + client.close(); + } + } + + @Test + void testWriteOnBulkFlush() throws Exception { + final String index = "test-bulk-flush-without-checkpoint"; + final int flushAfterNActions = 5; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + + // Ignore flush on checkpoint + writer.prepareCommit(false); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3, 4); + + // Trigger flush + writer.write(Tuple2.of(5, "test-5"), null); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5); + + writer.write(Tuple2.of(6, "test-6"), null); + context.assertThatIdsAreNotWritten(index, 6); + + // Force flush + writer.prepareCommit(true); + context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5, 6); + } + } + + @Test + void testWriteOnBulkIntervalFlush() throws Exception { + final String index = "test-bulk-flush-with-interval"; + + // Configure bulk processor to flush every 1s; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + writer.write(Tuple2.of(4, buildMessage(4)), null); + waitUntilCondition( + () -> { + try { + context.assertThatIdsAreWritten(index, 1, 2, 3, 4); + return true; + } catch (ElasticsearchStatusException e) { + return false; + } + }, + // Wait for at-least one flush to happen + Deadline.fromNow(Duration.ofSeconds(2))); + } + } + + @Test + void testWriteOnCheckpoint() throws Exception { + final String index = "test-bulk-flush-with-checkpoint"; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0); + + // Enable flush on checkpoint + try (final ElasticsearchWriter> writer = + createWriter(index, true, bulkProcessorConfig)) { + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.write(Tuple2.of(3, buildMessage(3)), null); + + context.assertThatIdsAreNotWritten(index, 1, 2, 3); + + // Trigger flush + writer.prepareCommit(false); + + context.assertThatIdsAreWritten(index, 1, 2, 3); + } + } + + @Test + void testIncrementByteOutMetric() throws Exception { + final String index = "test-inc-byte-out"; + final OperatorIOMetricGroup operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); + final InternalSinkWriterMetricGroup metricGroup = + InternalSinkWriterMetricGroup.mock( + metricListener.getMetricGroup(), operatorIOMetricGroup); + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig, metricGroup)) { + final Counter numBytesOut = operatorIOMetricGroup.getNumBytesOutCounter(); + assertEquals(numBytesOut.getCount(), 0); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + long first = numBytesOut.getCount(); + + assertTrue(first > 0); + + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + assertTrue(numBytesOut.getCount() > first); + } + } + + @Test + void testCurrentSendTime() throws Exception { + final String index = "test-current-send-time"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional> currentSendTime = + metricListener.getGauge("currentSendTime"); + writer.write(Tuple2.of(1, buildMessage(1)), null); + writer.write(Tuple2.of(2, buildMessage(2)), null); + + assertTrue(currentSendTime.isPresent()); + assertThat(currentSendTime.get().getValue(), greaterThan(0L)); + } + } + + private ElasticsearchWriter> createWriter( + String index, boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig) { + return createWriter( + index, + flushOnCheckpoint, + bulkProcessorConfig, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + } + + private ElasticsearchWriter> createWriter( + String index, + boolean flushOnCheckpoint, + BulkProcessorConfig bulkProcessorConfig, + SinkWriterMetricGroup metricGroup) { + return new ElasticsearchWriter<>( + Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), + TestEmitter.jsonEmitter(index, context.getDataFieldName()), + flushOnCheckpoint, + bulkProcessorConfig, + new NetworkClientConfig(null, null, null), + metricGroup, + new TestMailbox()); + } + + private static class TestMailbox implements MailboxExecutor { + + @Override + public void execute( + ThrowingRunnable command, + String descriptionFormat, + Object... descriptionArgs) { + try { + command.run(); + } catch (Exception e) { + throw new RuntimeException("Unexpected error", e); + } + } + + @Override + public void yield() throws InterruptedException, FlinkRuntimeException { + Thread.sleep(100); + } + + @Override + public boolean tryYield() throws FlinkRuntimeException { + return false; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java new file mode 100644 index 00000000..140d26b8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java @@ -0,0 +1,77 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class TestClient { + + private static final String DATA_FIELD_NAME = "data"; + private final RestHighLevelClient client; + + TestClient(RestHighLevelClient client) { + this.client = client; + } + + void assertThatIdsAreNotWritten(String index, int... ids) throws IOException { + for (final int id : ids) { + try { + final GetResponse response = + client.get( + new GetRequest(index, Integer.toString(id)), + RequestOptions.DEFAULT); + assertFalse( + response.isExists(), String.format("Id %s is unexpectedly present.", id)); + } catch (ElasticsearchStatusException e) { + assertEquals(404, e.status().getStatus()); + } + } + } + + void assertThatIdsAreWritten(String index, int... ids) + throws IOException, InterruptedException { + for (final int id : ids) { + GetResponse response; + do { + response = + client.get( + new GetRequest(index, Integer.toString(id)), + RequestOptions.DEFAULT); + Thread.sleep(10); + } while (response.isSourceEmpty()); + assertEquals(buildMessage(id), response.getSource().get(DATA_FIELD_NAME)); + } + } + + String getDataFieldName() { + return DATA_FIELD_NAME; + } + + static String buildMessage(int id) { + return "test-" + id; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java new file mode 100644 index 00000000..41d3a5eb --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -0,0 +1,75 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; + +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +class TestEmitter implements ElasticsearchEmitter> { + + private final String index; + private final XContentBuilderProvider xContentBuilderProvider; + private final String dataFieldName; + + public static TestEmitter jsonEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::jsonBuilder); + } + + public static TestEmitter smileEmitter(String index, String dataFieldName) { + return new TestEmitter(index, dataFieldName, XContentFactory::smileBuilder); + } + + private TestEmitter( + String index, String dataFieldName, XContentBuilderProvider xContentBuilderProvider) { + this.dataFieldName = dataFieldName; + this.index = index; + this.xContentBuilderProvider = xContentBuilderProvider; + } + + @Override + public void emit( + Tuple2 element, SinkWriter.Context context, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + + public IndexRequest createIndexRequest(Tuple2 element) { + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + try { + return new IndexRequest(index) + .id(element.f0.toString()) + .source(xContentBuilderProvider.getBuilder().map(document)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @FunctionalInterface + private interface XContentBuilderProvider extends Serializable { + XContentBuilder getBuilder() throws IOException; + } +} From 67dff2bcf932e282771f1b24a76226bb5b005abf Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 26 Oct 2021 00:48:06 +0200 Subject: [PATCH 155/207] [FLINK-24018][build] Remove Scala dependencies from Java APIs --- .../flink-connector-elasticsearch-base/pom.xml | 10 +++++----- .../flink-connector-elasticsearch6/pom.xml | 14 +++++++------- .../flink-connector-elasticsearch7/pom.xml | 14 +++++++------- .../flink-sql-connector-elasticsearch6/pom.xml | 4 ++-- .../flink-sql-connector-elasticsearch7/pom.xml | 4 ++-- 5 files changed, 23 insertions(+), 23 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 5a94cb9a..176986af 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-connector-elasticsearch-base_${scala.binary.version} + flink-connector-elasticsearch-base Flink : Connectors : Elasticsearch base jar @@ -46,7 +46,7 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} provided @@ -56,7 +56,7 @@ under the License. org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} + flink-table-api-java-bridge ${project.version} provided true @@ -87,7 +87,7 @@ under the License. org.apache.flink - flink-test-utils_${scala.binary.version} + flink-test-utils ${project.version} test @@ -102,7 +102,7 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} test test-jar diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 6790c36d..07082e88 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-connector-elasticsearch6_${scala.binary.version} + flink-connector-elasticsearch6 Flink : Connectors : Elasticsearch 6 jar @@ -46,7 +46,7 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} provided @@ -56,7 +56,7 @@ under the License. org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} + flink-table-api-java-bridge ${project.version} provided true @@ -66,7 +66,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} + flink-connector-elasticsearch-base ${project.version} @@ -96,14 +96,14 @@ under the License. org.apache.flink - flink-test-utils_${scala.binary.version} + flink-test-utils ${project.version} test org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} test test-jar @@ -111,7 +111,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} + flink-connector-elasticsearch-base ${project.version} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index de02b967..bfee4f89 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-connector-elasticsearch7_${scala.binary.version} + flink-connector-elasticsearch7 Flink : Connectors : Elasticsearch 7 jar @@ -46,7 +46,7 @@ under the License. org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} provided @@ -55,7 +55,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} + flink-connector-elasticsearch-base ${project.version} @@ -71,7 +71,7 @@ under the License. org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} + flink-table-api-java-bridge ${project.version} provided true @@ -94,14 +94,14 @@ under the License. org.apache.flink - flink-test-utils_${scala.binary.version} + flink-test-utils ${project.version} test org.apache.flink - flink-streaming-java_${scala.binary.version} + flink-streaming-java ${project.version} test test-jar @@ -109,7 +109,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch-base_${scala.binary.version} + flink-connector-elasticsearch-base ${project.version} diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index b0a09873..585e54e4 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-sql-connector-elasticsearch6_${scala.binary.version} + flink-sql-connector-elasticsearch6 Flink : Connectors : SQL : Elasticsearch 6 jar @@ -38,7 +38,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch6_${scala.binary.version} + flink-connector-elasticsearch6 ${project.version} diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 6fe470b9..12586efe 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -30,7 +30,7 @@ under the License. .. - flink-sql-connector-elasticsearch7_${scala.binary.version} + flink-sql-connector-elasticsearch7 Flink : Connectors : SQL : Elasticsearch 7 jar @@ -38,7 +38,7 @@ under the License. org.apache.flink - flink-connector-elasticsearch7_${scala.binary.version} + flink-connector-elasticsearch7 ${project.version} From 9ecf4ec624d9cc595556084fd7065086d963f266 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 22 Oct 2021 18:17:06 +0200 Subject: [PATCH 156/207] [FLINK-24599][table] Replace static methods with convenient methods in LogicalType Make the code of calling `hasRoot`/`hasFamily` less verbose by replacing the static utility methods with member methods `LogicalType#is()`. This closes #17550. --- .../elasticsearch/table/ElasticsearchValidationUtils.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java index d0657e61..6452d006 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -31,8 +31,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; - /** Utility methods for validating Elasticsearch properties. */ @Internal class ElasticsearchValidationUtils { @@ -70,8 +68,7 @@ public static void validatePrimaryKey(TableSchema schema) { schema.getFieldDataType(fieldName) .get() .getLogicalType(); - if (hasRoot( - logicalType, + if (logicalType.is( LogicalTypeRoot.DISTINCT_TYPE)) { return ((DistinctType) logicalType) .getSourceType() From 78119238a506216e9495e61b2a35060418144233 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 7 Oct 2021 11:00:57 +0200 Subject: [PATCH 157/207] [FLINK-24327][connectors/elasticsearch] Integrate unified Elasticsearch 7 sink with Table API --- .../table/ElasticsearchValidationUtils.java | 54 ++++ .../table/IndexGeneratorFactory.java | 15 + .../elasticsearch/table/KeyExtractor.java | 20 +- .../table/LogicalTypeWithIndex.java | 31 ++ .../table/SerializableFunction.java | 24 ++ .../sink/ElasticsearchSinkBuilder.java | 2 +- .../sink/ElasticsearchWriter.java | 1 + .../table/Elasticsearch7Configuration.java | 75 ++++- .../table/Elasticsearch7ConnectorOptions.java | 124 ++++++++ .../table/Elasticsearch7DynamicSink.java | 290 ++++------------- .../Elasticsearch7DynamicSinkFactory.java | 84 +++-- .../table/RowElasticsearchEmitter.java | 101 ++++++ .../Elasticsearch7DynamicSinkFactoryTest.java | 4 +- .../Elasticsearch7DynamicSinkITCase.java | 75 ++--- .../table/Elasticsearch7DynamicSinkTest.java | 294 ------------------ 15 files changed, 588 insertions(+), 606 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java index 6452d006..b2ea6ffe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; @@ -36,6 +37,7 @@ class ElasticsearchValidationUtils { private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + private static final Set ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); static { ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); @@ -46,6 +48,24 @@ class ElasticsearchValidationUtils { ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DATE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH); + ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME); } /** @@ -56,6 +76,7 @@ class ElasticsearchValidationUtils { * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link * LogicalTypeRoot#RAW} type. */ + @Deprecated public static void validatePrimaryKey(TableSchema schema) { schema.getPrimaryKey() .ifPresent( @@ -90,5 +111,38 @@ public static void validatePrimaryKey(TableSchema schema) { }); } + /** + * Checks that the table does not have a primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(DataType primaryKeyDataType) { + List fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType); + List illegalTypes = + fieldDataTypes.stream() + .map(DataType::getLogicalType) + .map( + logicalType -> { + if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(t -> !ALLOWED_PRIMARY_KEY_TYPES.contains(t)) + .collect(Collectors.toList()); + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.", + illegalTypes)); + } + } + private ElasticsearchValidationUtils() {} } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java index bf7ac64a..a0b8faf0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -63,6 +63,21 @@ final class IndexGeneratorFactory { private IndexGeneratorFactory() {} + public static IndexGenerator createIndexGenerator( + String index, List fieldNames, List dataTypes) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + fieldNames.toArray(new String[0]), + dataTypes.toArray(new DataType[0]), + indexHelper); + } else { + return new StaticIndexGenerator(index); + } + } + + @Deprecated public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { final IndexHelper indexHelper = new IndexHelper(); if (indexHelper.checkIsDynamicIndex(index)) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java index ae7c522b..fb833279 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -37,7 +37,7 @@ /** An extractor for a Elasticsearch key from a {@link RowData}. */ @Internal -class KeyExtractor implements Function, Serializable { +class KeyExtractor implements SerializableFunction { private final FieldFormatter[] fieldFormatters; private final String keyDelimiter; @@ -81,6 +81,7 @@ public int getIndex() { } } + @Deprecated public static Function createKeyExtractor( TableSchema schema, String keyDelimiter) { return schema.getPrimaryKey() @@ -109,6 +110,23 @@ public static Function createKeyExtractor( .orElseGet(() -> (Function & Serializable) (row) -> null); } + public static SerializableFunction createKeyExtractor( + List primaryKeyTypesWithIndex, String keyDelimiter) { + if (!primaryKeyTypesWithIndex.isEmpty()) { + FieldFormatter[] formatters = + primaryKeyTypesWithIndex.stream() + .map( + logicalTypeWithIndex -> + toFormatter( + logicalTypeWithIndex.index, + logicalTypeWithIndex.logicalType)) + .toArray(FieldFormatter[]::new); + return new KeyExtractor(formatters, keyDelimiter); + } else { + return (row) -> null; + } + } + private static FieldFormatter toFormatter(int index, LogicalType type) { switch (type.getTypeRoot()) { case DATE: diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java new file mode 100644 index 00000000..81c63475 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java @@ -0,0 +1,31 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.table.types.logical.LogicalType; + +class LogicalTypeWithIndex { + public final int index; + public final LogicalType logicalType; + + LogicalTypeWithIndex(int index, LogicalType logicalType) { + this.index = index; + this.logicalType = logicalType; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java new file mode 100644 index 00000000..be243134 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java @@ -0,0 +1,24 @@ +/* + * 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.connectors.elasticsearch.table; + +import java.io.Serializable; +import java.util.function.Function; + +interface SerializableFunction extends Function, Serializable {} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java index 68e30cb3..233af3b9 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java @@ -154,7 +154,7 @@ public ElasticsearchSinkBuilder setBulkFlushInterval(long intervalMillis) { } /** - * Sets the type of back of to use when flushing bulk requests. The default bulk flush back off + * Sets the type of back off to use when flushing bulk requests. The default bulk flush back off * type is {@link FlushBackoffType#NONE}. * *

Sets the amount of delay between each backoff attempt when flushing bulk requests, in diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index b024bb24..c17dc523 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -150,6 +150,7 @@ public List prepareCommit(boolean flush) throws IOException, InterruptedEx public void close() throws Exception { closed = true; emitter.close(); + bulkProcessor.close(); client.close(); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java index 6bd28cf4..276f77c0 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -20,20 +20,87 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.api.ValidationException; import org.apache.http.HttpHost; +import java.time.Duration; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.util.Preconditions.checkNotNull; /** Elasticsearch 7 specific configuration. */ @Internal -final class Elasticsearch7Configuration extends ElasticsearchConfiguration { - Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { - super(config, classLoader); +final class Elasticsearch7Configuration { + protected final ReadableConfig config; + + Elasticsearch7Configuration(ReadableConfig config) { + this.config = checkNotNull(config); + } + + public int getBulkFlushMaxActions() { + return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION); + } + + public long getBulkFlushMaxByteSize() { + return config.get(BULK_FLUSH_MAX_SIZE_OPTION).getBytes(); + } + + public long getBulkFlushInterval() { + return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + } + + public DeliveryGuarantee getDeliveryGuarantee() { + return config.get(DELIVERY_GUARANTEE_OPTION); + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public Optional getBulkFlushBackoffType() { + return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public String getIndex() { + return config.get(INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); } public List getHosts() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java new file mode 100644 index 00000000..6048e124 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.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.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; + +import java.time.Duration; +import java.util.List; + +/** Options for the Elasticsearch connector. */ +@PublicEvolving +public class Elasticsearch7ConnectorOptions { + + private Elasticsearch7ConnectorOptions() {} + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + + public static final ConfigOption BULK_FLUSH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(FlushBackoffType.class) + .noDefaultValue() + .withDescription("Backoff strategy"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + public static final ConfigOption DELIVERY_GUARANTEE_OPTION = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) + .withDescription("Optional delivery guarantee when committing."); +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index 0fe2dac4..e6a03dc3 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -19,89 +19,50 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; -import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; -import org.apache.flink.table.api.TableSchema; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilder; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.connector.sink.SinkProvider; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; import org.apache.flink.util.StringUtils; import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.impl.client.BasicCredentialsProvider; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.common.xcontent.XContentType; -import javax.annotation.Nullable; - import java.util.List; import java.util.Objects; +import static org.apache.flink.util.Preconditions.checkNotNull; + /** * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a * logical description. */ @Internal final class Elasticsearch7DynamicSink implements DynamicTableSink { - @VisibleForTesting - static final Elasticsearch7RequestFactory REQUEST_FACTORY = - new Elasticsearch7DynamicSink.Elasticsearch7RequestFactory(); private final EncodingFormat> format; - private final TableSchema schema; + private final DataType physicalRowDataType; + private final List primaryKeyLogicalTypesWithIndex; private final Elasticsearch7Configuration config; - public Elasticsearch7DynamicSink( - EncodingFormat> format, - Elasticsearch7Configuration config, - TableSchema schema) { - this(format, config, schema, (ElasticsearchSink.Builder::new)); - } - - // -------------------------------------------------------------- - // Hack to make configuration testing possible. - // - // The code in this block should never be used outside of tests. - // Having a way to inject a builder we can assert the builder in - // the test. We can not assert everything though, e.g. it is not - // possible to assert flushing on checkpoint, as it is configured - // on the sink itself. - // -------------------------------------------------------------- - - private final ElasticSearchBuilderProvider builderProvider; - - @FunctionalInterface - interface ElasticSearchBuilderProvider { - ElasticsearchSink.Builder createBuilder( - List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); - } - Elasticsearch7DynamicSink( EncodingFormat> format, Elasticsearch7Configuration config, - TableSchema schema, - ElasticSearchBuilderProvider builderProvider) { - this.format = format; - this.schema = schema; - this.config = config; - this.builderProvider = builderProvider; + List primaryKeyLogicalTypesWithIndex, + DataType physicalRowDataType) { + this.format = checkNotNull(format); + this.physicalRowDataType = checkNotNull(physicalRowDataType); + this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex); + this.config = checkNotNull(config); } - // -------------------------------------------------------------- - // End of hack to make configuration testing possible - // -------------------------------------------------------------- - @Override public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { ChangelogMode.Builder builder = ChangelogMode.newBuilder(); @@ -114,61 +75,59 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { } @Override - public SinkFunctionProvider getSinkRuntimeProvider(Context context) { - return () -> { - SerializationSchema format = - this.format.createRuntimeEncoder(context, schema.toRowDataType()); - - final RowElasticsearchSinkFunction upsertFunction = - new RowElasticsearchSinkFunction( - IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), - null, // this is deprecated in es 7+ - format, - XContentType.JSON, - REQUEST_FACTORY, - KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); - - final ElasticsearchSink.Builder builder = - builderProvider.createBuilder(config.getHosts(), upsertFunction); - - builder.setFailureHandler(config.getFailureHandler()); - builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); - builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); - builder.setBulkFlushInterval(config.getBulkFlushInterval()); - builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); - config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); - config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); - config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + SerializationSchema format = + this.format.createRuntimeEncoder(context, physicalRowDataType); + + final RowElasticsearchEmitter rowElasticsearchEmitter = + new RowElasticsearchEmitter( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), + DataType.getFieldNames(physicalRowDataType), + DataType.getFieldDataTypes(physicalRowDataType)), + format, + XContentType.JSON, + KeyExtractor.createKeyExtractor( + primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter())); + + final ElasticsearchSinkBuilder builder = ElasticsearchSink.builder(); + builder.setEmitter(rowElasticsearchEmitter); + builder.setHosts(config.getHosts().toArray(new HttpHost[0])); + builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + + if (config.getBulkFlushBackoffType().isPresent()) { + FlushBackoffType backoffType = config.getBulkFlushBackoffType().get(); + int backoffMaxRetries = config.getBulkFlushBackoffRetries().get(); + long backoffDelayMs = config.getBulkFlushBackoffDelay().get(); + + builder.setBulkFlushBackoffStrategy(backoffType, backoffMaxRetries, backoffDelayMs); + } - // we must overwrite the default factory which is defined with a lambda because of a bug - // in shading lambda serialization shading see FLINK-18006 - if (config.getUsername().isPresent() - && config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { - builder.setRestClientFactory( - new AuthRestClientFactory( - config.getPathPrefix().orElse(null), - config.getUsername().get(), - config.getPassword().get())); - } else { - builder.setRestClientFactory( - new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); - } + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + builder.setConnectionUsername(config.getUsername().get()); + } - final ElasticsearchSink sink = builder.build(); + if (config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setConnectionPassword(config.getPassword().get()); + } - if (config.isDisableFlushOnCheckpoint()) { - sink.disableFlushOnCheckpoint(); - } + if (config.getPathPrefix().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPathPrefix().get())) { + builder.setConnectionPathPrefix(config.getPathPrefix().get()); + } - return sink; - }; + return SinkProvider.of(builder.build()); } @Override public DynamicTableSink copy() { - return this; + return new Elasticsearch7DynamicSink( + format, config, primaryKeyLogicalTypesWithIndex, physicalRowDataType); } @Override @@ -176,126 +135,6 @@ public String asSummaryString() { return "Elasticsearch7"; } - /** Serializable {@link RestClientFactory} used by the sink. */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ - @VisibleForTesting - static class AuthRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - private final String username; - private final String password; - private transient CredentialsProvider credentialsProvider; - - public AuthRestClientFactory( - @Nullable String pathPrefix, String username, String password) { - this.pathPrefix = pathPrefix; - this.password = password; - this.username = username; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - if (credentialsProvider == null) { - credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials( - AuthScope.ANY, new UsernamePasswordCredentials(username, password)); - } - restClientBuilder.setHttpClientConfigCallback( - httpAsyncClientBuilder -> - httpAsyncClientBuilder.setDefaultCredentialsProvider( - credentialsProvider)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AuthRestClientFactory that = (AuthRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix) - && Objects.equals(username, that.username) - && Objects.equals(password, that.password); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix, password, username); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the - * sink. - */ - private static class Elasticsearch7RequestFactory implements RequestFactory { - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new IndexRequest(index).id(key).source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, key); - } - } - @Override public boolean equals(Object o) { if (this == o) { @@ -306,13 +145,14 @@ public boolean equals(Object o) { } Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; return Objects.equals(format, that.format) - && Objects.equals(schema, that.schema) - && Objects.equals(config, that.config) - && Objects.equals(builderProvider, that.builderProvider); + && Objects.equals(physicalRowDataType, that.physicalRowDataType) + && Objects.equals( + primaryKeyLogicalTypesWithIndex, that.primaryKeyLogicalTypesWithIndex) + && Objects.equals(config, that.config); } @Override public int hashCode() { - return Objects.hash(format, schema, config, builderProvider); + return Objects.hash(format, physicalRowDataType, primaryKeyLogicalTypesWithIndex, config); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index e5fed88c..0646d26f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -22,38 +22,41 @@ import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DynamicTableSinkFactory; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.util.StringUtils; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.USERNAME_OPTION; /** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ @Internal @@ -63,25 +66,46 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory private static final Set> optionalOptions = Stream.of( KEY_DELIMITER_OPTION, - FAILURE_HANDLER_OPTION, - FLUSH_ON_CHECKPOINT_OPTION, - BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_SIZE_OPTION, BULK_FLUSH_MAX_ACTIONS_OPTION, BULK_FLUSH_INTERVAL_OPTION, BULK_FLUSH_BACKOFF_TYPE_OPTION, BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, - CONNECTION_PATH_PREFIX, + CONNECTION_PATH_PREFIX_OPTION, FORMAT_OPTION, + DELIVERY_GUARANTEE_OPTION, PASSWORD_OPTION, USERNAME_OPTION) .collect(Collectors.toSet()); @Override public DynamicTableSink createDynamicTableSink(Context context) { - TableSchema tableSchema = context.getCatalogTable().getSchema(); - ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + DataType physicalRowDataType = context.getPhysicalRowDataType(); + int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); + if (primaryKeyIndexes.length != 0) { + DataType pkDataType = DataType.projectFields(physicalRowDataType, primaryKeyIndexes); + + ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); + } + + ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); + List primaryKeyLogicalTypesWithIndex = + Arrays.stream(primaryKeyIndexes) + .mapToObj( + index -> { + Optional column = resolvedSchema.getColumn(index); + if (!column.isPresent()) { + throw new IllegalStateException( + String.format( + "No primary key column found with index '%s'.", + index)); + } + LogicalType logicalType = + column.get().getDataType().getLogicalType(); + return new LogicalTypeWithIndex(index, logicalType); + }) + .collect(Collectors.toList()); final FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); @@ -92,17 +116,15 @@ public DynamicTableSink createDynamicTableSink(Context context) { helper.validate(); Configuration configuration = new Configuration(); context.getCatalogTable().getOptions().forEach(configuration::setString); - Elasticsearch7Configuration config = - new Elasticsearch7Configuration(configuration, context.getClassLoader()); + Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration); validate(config, configuration); return new Elasticsearch7DynamicSink( - format, config, TableSchemaUtils.getPhysicalSchema(tableSchema)); + format, config, primaryKeyLogicalTypesWithIndex, physicalRowDataType); } private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { - config.getFailureHandler(); // checks if we can instantiate the custom failure handler config.getHosts(); // validate hosts validate( config.getIndex().length() >= 1, @@ -121,9 +143,9 @@ private void validate(Elasticsearch7Configuration config, Configuration original () -> String.format( "'%s' must be in MB granularity. Got: %s", - BULK_FLASH_MAX_SIZE_OPTION.key(), + BULK_FLUSH_MAX_SIZE_OPTION.key(), originalConfiguration - .get(BULK_FLASH_MAX_SIZE_OPTION) + .get(BULK_FLUSH_MAX_SIZE_OPTION) .toHumanReadableString())); validate( config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java new file mode 100644 index 00000000..8c1f2081 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java @@ -0,0 +1,101 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchEmitter; +import org.apache.flink.connector.elasticsearch.sink.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ +class RowElasticsearchEmitter implements ElasticsearchEmitter { + + private final IndexGenerator indexGenerator; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final Function createKey; + + public RowElasticsearchEmitter( + IndexGenerator indexGenerator, + SerializationSchema serializationSchema, + XContentType contentType, + Function createKey) { + this.indexGenerator = checkNotNull(indexGenerator); + this.serializationSchema = checkNotNull(serializationSchema); + this.contentType = checkNotNull(contentType); + this.createKey = checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void emit(RowData element, SinkWriter.Context context, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + new UpdateRequest(indexGenerator.generate(row), key) + .doc(document, contentType) + .upsert(document, contentType); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + new IndexRequest(indexGenerator.generate(row)) + .id(key) + .source(document, contentType); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = new DeleteRequest(indexGenerator.generate(row), key); + indexer.add(deleteRequest); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 3ff21ed1..4f40c1cc 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -159,9 +159,7 @@ public void validatePrimaryKeyOnIllegalColumn() { thrown.expect(ValidationException.class); thrown.expectMessage( "The table has a primary key on columns of illegal types: " - + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" - + " Elasticsearch sink does not support primary keys on columns of types: " - + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY]."); sinkFactory.createDynamicTableSink( context() .withSchema( diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index fdf7a3a0..03d20366 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -29,7 +29,7 @@ import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.connector.sink.SinkProvider; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -111,35 +111,28 @@ public void testWritingDocuments() throws Exception { String index = "writing-documents"; Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - SinkFunctionProvider sinkRuntimeProvider = - (SinkFunctionProvider) - sinkFactory - .createDynamicTableSink( - context() - .withSchema(schema) - .withOption( - ElasticsearchConnectorOptions.INDEX_OPTION - .key(), - index) - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION - .key(), - elasticsearchContainer.getHttpHostAddress()) - .withOption( - ElasticsearchConnectorOptions - .FLUSH_ON_CHECKPOINT_OPTION - .key(), - "false") - .build()) - .getSinkRuntimeProvider(new MockContext()); - - SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + DynamicTableSink.SinkRuntimeProvider runtimeProvider = + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), + index) + .withOption( + Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + .build()) + .getSinkRuntimeProvider(new MockContext()); + + final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; + final Sink sink = sinkProvider.createSink(); StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); environment.setParallelism(4); rowData.setRowKind(RowKind.UPDATE_AFTER); - environment.fromElements(rowData).addSink(sinkFunction); + environment.fromElements(rowData).sinkTo(sink); environment.execute(); Client client = getClient(); @@ -178,15 +171,11 @@ public void testWritingDocumentsFromTableApi() throws Exception { + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) + Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") + Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + ")"); tableEnvironment @@ -236,15 +225,11 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) + Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") + Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + ")"); tableEnvironment @@ -326,15 +311,11 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + String.format( "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) + Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) + String.format( "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") + Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + ")"); tableEnvironment diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java deleted file mode 100644 index 6ab6828f..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; -import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.util.TestLogger; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.ActionRequest; -import org.junit.Test; -import org.mockito.Mockito; - -import java.util.List; - -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; - -/** Tests for {@link Elasticsearch7DynamicSink} parameters. */ -public class Elasticsearch7DynamicSinkTest extends TestLogger { - - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - private static final String HOSTNAME = "host1"; - private static final int PORT = 1234; - private static final String SCHEMA = "https"; - private static final String INDEX = "MyIndex"; - private static final String DOC_TYPE = "MyType"; - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = - new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration( - getConfig(), this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(true); - verify(provider.builderSpy) - .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - verify(provider.builderSpy).setBulkFlushBackoffDelay(123); - verify(provider.builderSpy).setBulkFlushBackoffRetries(3); - verify(provider.builderSpy).setBulkFlushInterval(100); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); - verify(provider.builderSpy) - .setRestClientFactory( - new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); - verify(provider.sinkSpy).disableFlushOnCheckpoint(); - } - - @Test - public void testDefaultConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = - new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration( - configuration, this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy) - .setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - @Test - public void testAuthConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch7DynamicSink testSink = - new Elasticsearch7DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch7Configuration( - configuration, this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy) - .setRestClientFactory( - new Elasticsearch7DynamicSink.AuthRestClientFactory( - null, USERNAME, PASSWORD)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - private Configuration getConfig() { - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString( - ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); - configuration.setString( - ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), - DummyFailureHandler.class.getName()); - configuration.setString( - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); - return configuration; - } - - private static class BuilderProvider - implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { - public ElasticsearchSink.Builder builderSpy; - public ElasticsearchSink sinkSpy; - - @Override - public ElasticsearchSink.Builder createBuilder( - List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { - builderSpy = - Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); - doAnswer( - invocation -> { - sinkSpy = - Mockito.spy( - (ElasticsearchSink) - invocation.callRealMethod()); - return sinkSpy; - }) - .when(builderSpy) - .build(); - - return builderSpy; - } - } - - private TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - private static class DummySerializationSchema implements SerializationSchema { - - private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); - - @Override - public byte[] serialize(RowData element) { - return new byte[0]; - } - } - - private static class DummyEncodingFormat - implements EncodingFormat> { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - return DummySerializationSchema.INSTANCE; - } - - @Override - public ChangelogMode getChangelogMode() { - return null; - } - } - - private static class MockSinkContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { - return null; - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType consumedDataType) { - return null; - } - } - - /** Custom failure handler for testing. */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure( - ActionRequest action, - Throwable failure, - int restStatusCode, - RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } -} From ed9f6da184c9261c2c81c2267c8834918a4a542d Mon Sep 17 00:00:00 2001 From: martijnvisser Date: Wed, 3 Nov 2021 09:26:29 +0100 Subject: [PATCH 158/207] [FLINK-24715][connectors][filesystems][formats] Update multiple Jackson dependencies to latest version --- .../src/main/resources/META-INF/NOTICE | 12 ++++++------ .../src/main/resources/META-INF/NOTICE | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index e28bbc8c..52ea8404 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -6,12 +6,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.12.1 -- com.fasterxml.jackson.core:jackson-databind:2.12.1 -- com.fasterxml.jackson.core:jackson-annotations:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 +- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 4617b41b..daf143bb 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -7,12 +7,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.carrotsearch:hppc:0.8.1 -- com.fasterxml.jackson.core:jackson-core:2.12.1 -- com.fasterxml.jackson.core:jackson-databind:2.12.1 -- com.fasterxml.jackson.core:jackson-annotations:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.12.1 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.12.1 +- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 - com.github.spullara.mustache.java:compiler:0.9.6 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 From 1216b5e7cd1e6e4884451dd136deeb2947257482 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 23 Nov 2021 14:52:55 +0100 Subject: [PATCH 159/207] [FLINK-24583][connectors/elasticsearch] Improve teststability by blocking until all records have been acknowledged by Elasticsearch --- .../sink/ElasticsearchWriter.java | 10 ++++++++ .../sink/ElasticsearchWriterITCase.java | 24 +++++++------------ 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index c17dc523..d47fb7f9 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -17,6 +17,7 @@ package org.apache.flink.connector.elasticsearch.sink; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.connector.sink.SinkWriter; import org.apache.flink.metrics.Counter; @@ -146,6 +147,15 @@ public List prepareCommit(boolean flush) throws IOException, InterruptedEx return Collections.emptyList(); } + @VisibleForTesting + void blockingFlushAllActions() throws InterruptedException { + while (pendingActions != 0) { + bulkProcessor.flush(); + LOG.info("Waiting for the response of {} pending actions.", pendingActions); + mailboxExecutor.yield(); + } + } + @Override public void close() throws Exception { closed = true; diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 91779dd1..9ea14db8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; @@ -33,7 +32,6 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.http.HttpHost; -import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; import org.junit.jupiter.api.AfterEach; @@ -48,12 +46,10 @@ import org.testcontainers.utility.DockerImageName; import java.io.IOException; -import java.time.Duration; import java.util.Collections; import java.util.Optional; import static org.apache.flink.connector.elasticsearch.sink.TestClient.buildMessage; -import static org.apache.flink.runtime.testutils.CommonTestUtils.waitUntilCondition; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -117,7 +113,7 @@ void testWriteOnBulkFlush() throws Exception { context.assertThatIdsAreNotWritten(index, 6); // Force flush - writer.prepareCommit(true); + writer.blockingFlushAllActions(); context.assertThatIdsAreWritten(index, 1, 2, 3, 4, 5, 6); } } @@ -136,18 +132,10 @@ void testWriteOnBulkIntervalFlush() throws Exception { writer.write(Tuple2.of(2, buildMessage(2)), null); writer.write(Tuple2.of(3, buildMessage(3)), null); writer.write(Tuple2.of(4, buildMessage(4)), null); - waitUntilCondition( - () -> { - try { - context.assertThatIdsAreWritten(index, 1, 2, 3, 4); - return true; - } catch (ElasticsearchStatusException e) { - return false; - } - }, - // Wait for at-least one flush to happen - Deadline.fromNow(Duration.ofSeconds(2))); + writer.blockingFlushAllActions(); } + + context.assertThatIdsAreWritten(index, 1, 2, 3, 4); } @Test @@ -191,6 +179,7 @@ void testIncrementByteOutMetric() throws Exception { writer.write(Tuple2.of(1, buildMessage(1)), null); writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.blockingFlushAllActions(); long first = numBytesOut.getCount(); assertTrue(first > 0); @@ -198,6 +187,7 @@ void testIncrementByteOutMetric() throws Exception { writer.write(Tuple2.of(1, buildMessage(1)), null); writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.blockingFlushAllActions(); assertTrue(numBytesOut.getCount() > first); } } @@ -216,6 +206,8 @@ void testCurrentSendTime() throws Exception { writer.write(Tuple2.of(1, buildMessage(1)), null); writer.write(Tuple2.of(2, buildMessage(2)), null); + writer.blockingFlushAllActions(); + assertTrue(currentSendTime.isPresent()); assertThat(currentSendTime.get().getValue(), greaterThan(0L)); } From 42209e2638dfa2bbf95116f1ad9dea4c9cd7b458 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 25 Nov 2021 11:55:32 +0100 Subject: [PATCH 160/207] [FLINK-25039][build] Disable shading of test jar by default --- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 1 - flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 1 - 2 files changed, 2 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 585e54e4..c747e0c0 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -56,7 +56,6 @@ under the License. shade - false *:* diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 12586efe..a5ba0abc 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -56,7 +56,6 @@ under the License. shade - false *:* From 154fffef7ae24f11f17f9329f79b221fe34eb05a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Wed, 20 Oct 2021 15:29:38 +0200 Subject: [PATCH 161/207] [FLINK-24325][connectors/elasticsearch] Remove Elasticsearch 5 from connectors --- .../pom.xml | 16 +- .../ElasticsearchApiCallBridge.java | 7 +- .../index/AbstractTimeIndexGenerator.java | 38 --- .../elasticsearch/index/IndexGenerator.java | 38 --- .../index/IndexGeneratorBase.java | 49 --- .../index/IndexGeneratorFactory.java | 282 ------------------ .../index/StaticIndexGenerator.java | 33 -- .../elasticsearch/table/IndexGenerator.java | 2 +- .../table/IndexGeneratorFactory.java | 12 - .../util/ElasticsearchUtils.java | 53 ---- .../ElasticsearchSinkBaseTest.java | 9 + .../elasticsearch/TestRequestIndexer.java} | 15 +- .../index/IndexGeneratorTest.java | 260 ---------------- .../table/IndexGeneratorFactoryTest.java | 215 ------------- .../table/IndexGeneratorTest.java | 264 ++++++++++++++++ .../flink-connector-elasticsearch6/pom.xml | 8 + 16 files changed, 302 insertions(+), 999 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java rename flink-connectors/flink-connector-elasticsearch-base/src/{main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java => test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java} (85%) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 176986af..9219775a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 5.3.3 + 7.5.1 @@ -85,6 +85,20 @@ under the License. + + + org.elasticsearch.client + transport + ${elasticsearch.version} + test + + + + org.testcontainers + elasticsearch + test + + org.apache.flink flink-test-utils diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 54e8399b..79ab9052 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -100,13 +100,10 @@ void configureBulkProcessorBackoff( * Creates a {@link RequestIndexer} that is able to work with {@link BulkProcessor} binary * compatible. */ - default RequestIndexer createBulkProcessorIndexer( + RequestIndexer createBulkProcessorIndexer( BulkProcessor bulkProcessor, boolean flushOnCheckpoint, - AtomicLong numPendingRequestsRef) { - return new PreElasticsearch6BulkProcessorIndexer( - bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); - } + AtomicLong numPendingRequestsRef); /** Perform any necessary state cleanup. */ default void cleanup() { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java deleted file mode 100644 index 1625ef5c..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/AbstractTimeIndexGenerator.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import java.time.format.DateTimeFormatter; - -/** Abstract class for time related {@link IndexGenerator}. */ -public abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { - - private final String dateTimeFormat; - protected transient DateTimeFormatter dateTimeFormatter; - - public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { - super(index); - this.dateTimeFormat = dateTimeFormat; - } - - @Override - public void open() { - this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java deleted file mode 100644 index 5f390f23..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGenerator.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.types.Row; - -import java.io.Serializable; - -/** This interface is responsible to generate index name from given {@link Row} record. */ -@Internal -public interface IndexGenerator extends Serializable { - - /** - * Initialize the index generator, this will be called only once before {@link #generate(Row)} - * is called. - */ - default void open() {} - - /** Generate index name according the the given row. */ - String generate(Row row); -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java deleted file mode 100644 index 1e08fa0c..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorBase.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import java.util.Objects; - -/** Base class for {@link IndexGenerator}. */ -public abstract class IndexGeneratorBase implements IndexGenerator { - - private static final long serialVersionUID = 1L; - protected final String index; - - public IndexGeneratorBase(String index) { - this.index = index; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof IndexGeneratorBase)) { - return false; - } - IndexGeneratorBase that = (IndexGeneratorBase) o; - return index.equals(that.index); - } - - @Override - public int hashCode() { - return Objects.hash(index); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java deleted file mode 100644 index bd84d9b3..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorFactory.java +++ /dev/null @@ -1,282 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.flink.types.Row; - -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -/** - * Factory of {@link IndexGenerator}. - * - *

Flink supports both static index and dynamic index. - * - *

If you want to have a static index, this option value should be a plain string, e.g. - * 'myusers', all the records will be consistently written into "myusers" index. - * - *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in - * the record to dynamically generate a target index. You can also use - * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the - * format specified by date_format_string. The date_format_string is compatible with {@link - * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', - * then a record with log_ts field value 2020-03-27 12:25:55 will be written into - * "myusers-2020-03-27" index. - */ -@Internal -public class IndexGeneratorFactory { - - private IndexGeneratorFactory() {} - - public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { - final IndexHelper indexHelper = new IndexHelper(); - if (indexHelper.checkIsDynamicIndex(index)) { - return createRuntimeIndexGenerator( - index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); - } else { - return new StaticIndexGenerator(index); - } - } - - private static IndexGenerator createRuntimeIndexGenerator( - String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) { - final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); - final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); - final String indexSuffix = - index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); - - final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); - final int indexFieldPos = - indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); - final TypeInformation indexFieldType = - TypeConversions.fromDataTypeToLegacyInfo(fieldTypes[indexFieldPos]); - - // validate index field type - indexHelper.validateIndexFieldType(indexFieldType); - - // time extract dynamic index pattern - if (isDynamicIndexWithFormat) { - final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldType); - // DataTypes.SQL_TIMESTAMP - if (indexFieldType == Types.LOCAL_DATE_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalDateTime indexField = (LocalDateTime) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else if (indexFieldType == Types.SQL_TIMESTAMP) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Timestamp indexField = (Timestamp) row.getField(indexFieldPos); - String indexFieldValueStr = - indexField.toLocalDateTime().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } - // DataTypes.SQL_DATE - else if (indexFieldType == Types.LOCAL_DATE) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalDate indexField = (LocalDate) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else if (indexFieldType == Types.SQL_DATE) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Date indexField = (Date) row.getField(indexFieldPos); - String indexFieldValueStr = - indexField.toLocalDate().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } // DataTypes.TIME - else if (indexFieldType == Types.LOCAL_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - LocalTime indexField = (LocalTime) row.getField(indexFieldPos); - String indexFieldValueStr = indexField.format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else if (indexFieldType == Types.SQL_TIME) { - return new AbstractTimeIndexGenerator(index, dateTimeFormat) { - @Override - public String generate(Row row) { - Time indexField = (Time) row.getField(indexFieldPos); - String indexFieldValueStr = - indexField.toLocalTime().format(dateTimeFormatter); - return indexPrefix.concat(indexFieldValueStr).concat(indexSuffix); - } - }; - } else { - throw new TableException( - String.format( - "Unsupported type '%s' found in Elasticsearch dynamic index field, " - + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", - TypeConversions.fromLegacyInfoToDataType(indexFieldType))); - } - } - // general dynamic index pattern - return new IndexGeneratorBase(index) { - @Override - public String generate(Row row) { - Object indexField = row.getField(indexFieldPos); - return indexPrefix - .concat(indexField == null ? "null" : indexField.toString()) - .concat(indexSuffix); - } - }; - } - - /** - * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field - * type ans parse index format from pattern. - */ - private static class IndexHelper { - private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); - private static final Pattern dynamicIndexTimeExtractPattern = - Pattern.compile(".*\\{.+\\|.*\\}.*"); - private static final List supportedTypes = new ArrayList<>(); - private static final Map defaultFormats = new HashMap<>(); - - static { - // time related types - supportedTypes.add(Types.LOCAL_DATE_TIME); - supportedTypes.add(Types.SQL_TIMESTAMP); - supportedTypes.add(Types.LOCAL_DATE); - supportedTypes.add(Types.SQL_DATE); - supportedTypes.add(Types.LOCAL_TIME); - supportedTypes.add(Types.SQL_TIME); - // general types - supportedTypes.add(Types.STRING); - supportedTypes.add(Types.SHORT); - supportedTypes.add(Types.INT); - supportedTypes.add(Types.LONG); - } - - static { - defaultFormats.put(Types.LOCAL_DATE_TIME, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(Types.SQL_TIMESTAMP, "yyyy_MM_dd_HH_mm_ss"); - defaultFormats.put(Types.LOCAL_DATE, "yyyy_MM_dd"); - defaultFormats.put(Types.SQL_DATE, "yyyy_MM_dd"); - defaultFormats.put(Types.LOCAL_TIME, "HH_mm_ss"); - defaultFormats.put(Types.SQL_TIME, "HH_mm_ss"); - } - - /** Validate the index field Type. */ - void validateIndexFieldType(TypeInformation indexTypeInfo) { - if (!supportedTypes.contains(indexTypeInfo)) { - throw new IllegalArgumentException( - String.format( - "Unsupported type %s of index field, " + "Supported types are: %s", - indexTypeInfo, supportedTypes)); - } - } - - /** Get the default date format. */ - String getDefaultFormat(TypeInformation indexTypeInfo) { - return defaultFormats.get(indexTypeInfo); - } - - /** Check general dynamic index is enabled or not by index pattern. */ - boolean checkIsDynamicIndex(String index) { - final Matcher matcher = dynamicIndexPattern.matcher(index); - int count = 0; - while (matcher.find()) { - count++; - } - if (count > 1) { - throw new TableException( - String.format( - "Chaining dynamic index pattern %s is not supported," - + " only support single dynamic index pattern.", - index)); - } - return count == 1; - } - - /** Check time extract dynamic index is enabled or not by index pattern. */ - boolean checkIsDynamicIndexWithFormat(String index) { - return dynamicIndexTimeExtractPattern.matcher(index).matches(); - } - - /** Extract dynamic index pattern string from index pattern string. */ - String extractDynamicIndexPatternStr(String index) { - int start = index.indexOf("{"); - int end = index.lastIndexOf("}"); - return index.substring(start, end + 1); - } - - /** Extract index field position in a fieldNames, return the field position. */ - int extractIndexFieldPos( - String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { - List fieldList = Arrays.asList(fieldNames); - String indexFieldName; - if (isDynamicIndexWithFormat) { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); - } else { - indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); - } - if (!fieldList.contains(indexFieldName)) { - throw new TableException( - String.format( - "Unknown field '%s' in index pattern '%s', please check the field name.", - indexFieldName, index)); - } - return fieldList.indexOf(indexFieldName); - } - - /** Extract dateTime format by the date format that extracted from index pattern string. */ - private String extractDateFormat(String index, TypeInformation indexTypeInfo) { - String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); - if ("".equals(format)) { - format = getDefaultFormat(indexTypeInfo); - } - return format; - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java deleted file mode 100644 index b0564fe3..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/index/StaticIndexGenerator.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import org.apache.flink.types.Row; - -/** A static {@link IndexGenerator} which generate fixed index name. */ -public class StaticIndexGenerator extends IndexGeneratorBase { - - public StaticIndexGenerator(String index) { - super(index); - } - - public String generate(Row row) { - return index; - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java index 636f3409..f0f0a46b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -34,6 +34,6 @@ interface IndexGenerator extends Serializable { */ default void open() {} - /** Generate index name according the the given row. */ + /** Generate index name according to the given row. */ String generate(RowData row); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java index a0b8faf0..2cd903d3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; @@ -77,17 +76,6 @@ public static IndexGenerator createIndexGenerator( } } - @Deprecated - public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { - final IndexHelper indexHelper = new IndexHelper(); - if (indexHelper.checkIsDynamicIndex(index)) { - return createRuntimeIndexGenerator( - index, schema.getFieldNames(), schema.getFieldDataTypes(), indexHelper); - } else { - return new StaticIndexGenerator(index); - } - } - interface DynamicFormatter extends Serializable { String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java deleted file mode 100644 index e311b09a..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/ElasticsearchUtils.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.connectors.elasticsearch.util; - -import org.apache.flink.annotation.Internal; - -import org.elasticsearch.common.transport.InetSocketTransportAddress; -import org.elasticsearch.common.transport.TransportAddress; - -import java.net.InetSocketAddress; -import java.util.ArrayList; -import java.util.List; - -/** Suite of utility methods for Elasticsearch. */ -@Internal -public class ElasticsearchUtils { - - /** - * Utility method to convert a {@link List} of {@link InetSocketAddress} to Elasticsearch {@link - * TransportAddress}. - * - * @param inetSocketAddresses The list of {@link InetSocketAddress} to convert. - */ - public static List convertInetSocketAddresses( - List inetSocketAddresses) { - if (inetSocketAddresses == null) { - return null; - } else { - List converted; - converted = new ArrayList<>(inetSocketAddresses.size()); - for (InetSocketAddress address : inetSocketAddresses) { - converted.add(new InetSocketTransportAddress(address)); - } - return converted; - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 285a21a7..67999916 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -48,6 +48,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import static org.mockito.Matchers.any; import static org.mockito.Mockito.doAnswer; @@ -676,6 +677,14 @@ public void configureBulkProcessorBackoff( public void verifyClientConnection(Client client) { // no need for this in the test cases here } + + @Override + public RequestIndexer createBulkProcessorIndexer( + BulkProcessor bulkProcessor, + boolean flushOnCheckpoint, + AtomicLong numPendingRequestsRef) { + return new TestRequestIndexer(bulkProcessor, flushOnCheckpoint, numPendingRequestsRef); + } } private static class SimpleSinkFunction implements ElasticsearchSinkFunction { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java similarity index 85% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java index 50801abf..dc0c146e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/PreElasticsearch6BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -18,8 +18,6 @@ package org.apache.flink.streaming.connectors.elasticsearch; -import org.apache.flink.annotation.Internal; - import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.delete.DeleteRequest; @@ -33,20 +31,13 @@ /** * Implementation of a {@link RequestIndexer}, using a {@link BulkProcessor}. {@link ActionRequest * ActionRequests} will be buffered before sending a bulk request to the Elasticsearch cluster. - * - * @deprecated This class is not binary compatible with newer Elasticsearch 6+ versions (i.e. the - * {@link #add(UpdateRequest...)} ). However, this module is currently compiled against a very - * old Elasticsearch version. */ -@Deprecated -@Internal -class PreElasticsearch6BulkProcessorIndexer implements RequestIndexer { - +class TestRequestIndexer implements RequestIndexer { private final BulkProcessor bulkProcessor; private final boolean flushOnCheckpoint; private final AtomicLong numPendingRequestsRef; - PreElasticsearch6BulkProcessorIndexer( + TestRequestIndexer( BulkProcessor bulkProcessor, boolean flushOnCheckpoint, AtomicLong numPendingRequestsRef) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java deleted file mode 100644 index 522788f5..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/index/IndexGeneratorTest.java +++ /dev/null @@ -1,260 +0,0 @@ -/* - * 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.connectors.elasticsearch.index; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.DataType; -import org.apache.flink.types.Row; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.temporal.UnsupportedTemporalTypeException; -import java.util.ArrayList; -import java.util.List; - -/** Suite tests for {@link IndexGenerator}. */ -public class IndexGeneratorTest { - - private TableSchema schema; - private List rows; - - @Before - public void prepareData() { - String[] fieldNames = - new String[] { - "id", - "item", - "log_ts", - "log_date", - "order_timestamp", - "log_time", - "local_datetime", - "local_date", - "local_time", - "note", - "status" - }; - DataType[] dataTypes = - new DataType[] { - DataTypes.INT(), - DataTypes.STRING(), - DataTypes.BIGINT(), - DataTypes.DATE().bridgedTo(java.sql.Date.class), - DataTypes.TIMESTAMP().bridgedTo(java.sql.Timestamp.class), - DataTypes.TIME().bridgedTo(java.sql.Time.class), - DataTypes.TIMESTAMP().bridgedTo(java.time.LocalDateTime.class), - DataTypes.DATE().bridgedTo(java.time.LocalDate.class), - DataTypes.TIME().bridgedTo(java.time.LocalTime.class), - DataTypes.STRING(), - DataTypes.BOOLEAN() - }; - schema = new TableSchema.Builder().fields(fieldNames, dataTypes).build(); - - rows = new ArrayList<>(); - rows.add( - Row.of( - 1, - "apple", - Timestamp.valueOf("2020-03-18 12:12:14").getTime(), - Date.valueOf("2020-03-18"), - Timestamp.valueOf("2020-03-18 12:12:14"), - Time.valueOf("12:12:14"), - LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000), - LocalDate.of(2020, 3, 18), - LocalTime.of(12, 13, 14, 2000), - "test1", - true)); - rows.add( - Row.of( - 2, - "peanut", - Timestamp.valueOf("2020-03-19 12:22:14").getTime(), - Date.valueOf("2020-03-19"), - Timestamp.valueOf("2020-03-19 12:22:21"), - Time.valueOf("12:22:21"), - LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000), - LocalDate.of(2020, 3, 19), - LocalTime.of(12, 13, 14, 2000), - "test2", - false)); - } - - @Test - public void testDynamicIndexFromTimestamp() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = - IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalDateTime() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "{local_datetime|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = - IndexGeneratorFactory.createIndexGenerator( - "{local_datetime|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromDate() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalDate() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromTime() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromLocalTime() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{local_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormat() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{local_time|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testGeneralDynamicIndex() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); - indexGenerator.open(); - Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testStaticIndex() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index", schema); - indexGenerator.open(); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testUnknownField() { - String expectedExceptionMsg = - "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," - + " please check the field name."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(e.getMessage(), expectedExceptionMsg); - } - } - - @Test - public void testUnsupportedTimeType() { - String expectedExceptionMsg = - "Unsupported type 'INT' found in Elasticsearch dynamic index field, " - + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedMultiParametersType() { - String expectedExceptionMsg = - "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," - + " only support single dynamic index pattern."; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date}-{local_time}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testDynamicIndexUnsupportedFormat() { - String expectedExceptionMsg = "Unsupported field: HourOfDay"; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date|yyyy/MM/dd HH:mm}", schema); - } catch (UnsupportedTemporalTypeException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedIndexFieldType() { - String expectedExceptionMsg = - "Unsupported type Boolean of index field, Supported types are:" - + " [LocalDateTime, Timestamp, LocalDate, Date, LocalTime, Time, String, Short, Integer, Long]"; - try { - IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); - } catch (IllegalArgumentException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java deleted file mode 100644 index 081864fe..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java +++ /dev/null @@ -1,215 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.util.TestLogger; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.sql.Timestamp; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.temporal.UnsupportedTemporalTypeException; -import java.util.ArrayList; -import java.util.List; - -/** Tests for {@link IndexGeneratorFactory}. */ -public class IndexGeneratorFactoryTest extends TestLogger { - - private TableSchema schema; - private List rows; - - @Before - public void prepareData() { - schema = - new TableSchema.Builder() - .field("id", DataTypes.INT()) - .field("item", DataTypes.STRING()) - .field("log_ts", DataTypes.BIGINT()) - .field("log_date", DataTypes.DATE()) - .field("log_time", DataTypes.TIME()) - .field("order_timestamp", DataTypes.TIMESTAMP()) - .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("status", DataTypes.BOOLEAN()) - .build(); - - rows = new ArrayList<>(); - rows.add( - GenericRowData.of( - 1, - StringData.fromString("apple"), - Timestamp.valueOf("2020-03-18 12:12:14").getTime(), - (int) LocalDate.parse("2020-03-18").toEpochDay(), - (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), - TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), - true)); - rows.add( - GenericRowData.of( - 2, - StringData.fromString("peanut"), - Timestamp.valueOf("2020-03-19 12:12:14").getTime(), - (int) LocalDate.parse("2020-03-19").toEpochDay(), - (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), - TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), - TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), - false)); - } - - @Test - public void testDynamicIndexFromTimestamp() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); - indexGenerator.open(); - Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); - IndexGenerator indexGenerator1 = - IndexGeneratorFactory.createIndexGenerator( - "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); - indexGenerator1.open(); - Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromDate() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexFromTime() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormat() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); - indexGenerator.open(); - Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testGeneralDynamicIndex() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); - indexGenerator.open(); - Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testStaticIndex() { - IndexGenerator indexGenerator = - IndexGeneratorFactory.createIndexGenerator("my-index", schema); - indexGenerator.open(); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); - } - - @Test - public void testUnknownField() { - String expectedExceptionMsg = - "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," - + " please check the field name."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(e.getMessage(), expectedExceptionMsg); - } - } - - @Test - public void testUnsupportedTimeType() { - String expectedExceptionMsg = - "Unsupported type 'INT' found in Elasticsearch dynamic index field, " - + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; - try { - IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedMultiParametersType() { - String expectedExceptionMsg = - "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," - + " only support single dynamic index pattern."; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{local_date}-{local_time}", schema); - } catch (TableException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testDynamicIndexUnsupportedFormat() { - String expectedExceptionMsg = "Unsupported field: HourOfDay"; - try { - IndexGeneratorFactory.createIndexGenerator( - "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); - } catch (UnsupportedTemporalTypeException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } - - @Test - public void testUnsupportedIndexFieldType() { - String expectedExceptionMsg = - "Unsupported type BOOLEAN of index field, Supported types are:" - + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," - + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; - try { - IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); - } catch (IllegalArgumentException e) { - Assert.assertEquals(expectedExceptionMsg, e.getMessage()); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java new file mode 100644 index 00000000..7c8db881 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java @@ -0,0 +1,264 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.Arrays; +import java.util.List; + +/** Suite tests for {@link IndexGenerator}. */ +public class IndexGeneratorTest { + + private static final List fieldNames = + Arrays.asList( + "id", + "item", + "log_ts", + "log_date", + "order_timestamp", + "log_time", + "local_datetime", + "local_date", + "local_time", + "note", + "status"); + + private static final List dataTypes = + Arrays.asList( + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.DATE().bridgedTo(Date.class), + DataTypes.TIMESTAMP().bridgedTo(Timestamp.class), + DataTypes.TIME().bridgedTo(Time.class), + DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class), + DataTypes.DATE().bridgedTo(LocalDate.class), + DataTypes.TIME().bridgedTo(LocalTime.class), + DataTypes.STRING(), + DataTypes.BOOLEAN()); + + private static final List rows = + Arrays.asList( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) Date.valueOf("2020-03-18").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-18 12:12:14")), + (int) + (Time.valueOf("12:12:14").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)), + (int) LocalDate.of(2020, 3, 18).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test1", + true), + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:22:14").getTime(), + (int) Date.valueOf("2020-03-19").toLocalDate().toEpochDay(), + TimestampData.fromTimestamp(Timestamp.valueOf("2020-03-19 12:22:21")), + (int) + (Time.valueOf("12:22:21").toLocalTime().toNanoOfDay() + / 1_000_000L), + TimestampData.fromLocalDateTime( + LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)), + (int) LocalDate.of(2020, 3, 19).toEpochDay(), + (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + "test2", + false)); + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDateTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH-ss}_index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{local_datetime|yyyy_MM_dd_HH_mm}_index", fieldNames, dataTypes); + indexGenerator1.open(); + Assertions.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromLocalTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|HH-mm}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12-13", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_time|}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", fieldNames, dataTypes); + indexGenerator.open(); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{unknown_ts|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{id|yyyy-MM-dd}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", fieldNames, dataTypes); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date|yyyy/MM/dd HH:mm}", fieldNames, dataTypes); + } catch (UnsupportedTemporalTypeException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", fieldNames, dataTypes); + } catch (IllegalArgumentException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 07082e88..a5e25f76 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -74,6 +74,10 @@ under the License. org.elasticsearch elasticsearch + + org.elasticsearch.client + elasticsearch-rest-high-level-client + @@ -118,6 +122,10 @@ under the License. org.elasticsearch elasticsearch + + org.elasticsearch.client + elasticsearch-rest-high-level-client + test-jar test From 3cbe86021a41d5d55a5983564be629256e0dedc2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 21 Oct 2021 12:19:18 +0200 Subject: [PATCH 162/207] [FLINK-24325][connectors/elasticsearch] Extract common unified Elasticsearch sink for versions 6 and 7 --- .../pom.xml | 13 +++ .../sink/BulkProcessorConfig.java | 9 +- .../sink/BulkRequestConsumerFactory.java | 41 +++++++ .../sink/ElasticsearchEmitter.java | 0 .../elasticsearch/sink/ElasticsearchSink.java | 12 +-- .../sink/ElasticsearchSinkBuilderBase.java} | 101 ++++++++++-------- .../sink/ElasticsearchWriter.java | 9 +- .../elasticsearch/sink/FlushBackoffType.java | 0 .../sink/NetworkClientConfig.java | 0 .../elasticsearch/sink/RequestIndexer.java | 0 .../sink/ElasticsearchSinkBaseITCase.java} | 66 ++++++------ .../ElasticsearchSinkBuilderBaseTest.java} | 42 ++++---- .../sink/ElasticsearchWriterITCase.java | 68 ++++++++++-- .../elasticsearch/sink/TestClientBase.java} | 21 ++-- .../elasticsearch/sink/TestEmitter.java | 3 + .../flink-connector-elasticsearch6/pom.xml | 7 ++ .../sink/Elasticsearch6SinkBuilder.java | 54 ++++++++++ .../sink/Elasticsearch6SinkBuilderTest.java | 36 +++++++ .../sink/Elasticsearch6SinkITCase.java | 55 ++++++++++ .../sink/Elasticsearch6TestClient.java | 36 +++++++ .../flink-connector-elasticsearch7/pom.xml | 7 ++ .../sink/Elasticsearch7SinkBuilder.java | 58 ++++++++++ .../table/Elasticsearch7DynamicSink.java | 4 +- .../sink/Elasticsearch7SinkBuilderTest.java | 36 +++++++ .../sink/Elasticsearch7SinkITCase.java | 55 ++++++++++ .../sink/Elasticsearch7TestClient.java | 37 +++++++ 26 files changed, 621 insertions(+), 149 deletions(-) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java (86%) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java (100%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java (92%) rename flink-connectors/{flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java => flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java} (74%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java (96%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java (100%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java (100%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java (100%) rename flink-connectors/{flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java => flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java} (81%) rename flink-connectors/{flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java => flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java} (59%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java (80%) rename flink-connectors/{flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java => flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java} (78%) rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java (95%) create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 9219775a..bff4b096 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -83,6 +83,12 @@ under the License. + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + @@ -106,6 +112,13 @@ under the License. test + + org.apache.flink + flink-connector-testing + ${project.version} + test + + org.apache.flink flink-runtime diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java similarity index 86% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java index 8ec88b6c..4ebe852a 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java @@ -29,6 +29,7 @@ class BulkProcessorConfig implements Serializable { private final FlushBackoffType flushBackoffType; private final int bulkFlushBackoffRetries; private final long bulkFlushBackOffDelay; + private final BulkRequestConsumerFactory bulkRequestConsumerFactory; BulkProcessorConfig( int bulkFlushMaxActions, @@ -36,13 +37,15 @@ class BulkProcessorConfig implements Serializable { long bulkFlushInterval, FlushBackoffType flushBackoffType, int bulkFlushBackoffRetries, - long bulkFlushBackOffDelay) { + long bulkFlushBackOffDelay, + BulkRequestConsumerFactory bulkRequestConsumerFactory) { this.bulkFlushMaxActions = bulkFlushMaxActions; this.bulkFlushMaxMb = bulkFlushMaxMb; this.bulkFlushInterval = bulkFlushInterval; this.flushBackoffType = checkNotNull(flushBackoffType); this.bulkFlushBackoffRetries = bulkFlushBackoffRetries; this.bulkFlushBackOffDelay = bulkFlushBackOffDelay; + this.bulkRequestConsumerFactory = bulkRequestConsumerFactory; } public int getBulkFlushMaxActions() { @@ -68,4 +71,8 @@ public int getBulkFlushBackoffRetries() { public long getBulkFlushBackOffDelay() { return bulkFlushBackOffDelay; } + + public BulkRequestConsumerFactory getBulkRequestConsumerFactory() { + return bulkRequestConsumerFactory; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java new file mode 100644 index 00000000..e8b45319 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java @@ -0,0 +1,41 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.Internal; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.Serializable; +import java.util.function.BiConsumer; + +/** + * {@link BulkRequestConsumerFactory} is used to bridge incompatible Elasticsearch Java API calls + * across different Elasticsearch versions. + */ +@Internal +interface BulkRequestConsumerFactory extends Serializable { + BulkRequestFactory create(RestHighLevelClient client); + + interface BulkRequestFactory + extends Serializable, BiConsumer> {} +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java similarity index 92% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index 37151f79..b1bf79de 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -51,7 +51,7 @@ * * * @param type of the records converted to Elasticsearch actions - * @see ElasticsearchSinkBuilder on how to construct a ElasticsearchSink + * @see ElasticsearchSinkBuilderBase on how to construct a ElasticsearchSink */ @PublicEvolving public class ElasticsearchSink implements Sink { @@ -76,16 +76,6 @@ public class ElasticsearchSink implements Sink { this.networkClientConfig = checkNotNull(networkClientConfig); } - /** - * Create a {@link ElasticsearchSinkBuilder} to construct a new {@link ElasticsearchSink}. - * - * @param type of incoming records - * @return {@link ElasticsearchSinkBuilder} - */ - public static ElasticsearchSinkBuilder builder() { - return new ElasticsearchSinkBuilder<>(); - } - @Override public SinkWriter createWriter(InitContext context, List states) throws IOException { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java similarity index 74% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index 233af3b9..979cab21 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -26,28 +26,17 @@ import java.util.Arrays; import java.util.List; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.apache.flink.util.Preconditions.checkState; /** - * Builder to construct a {@link ElasticsearchSink}. - * - *

The following example shows the minimal setup to create a ElasticsearchSink that submits - * actions on checkpoint or the default number of actions was buffered (1000). - * - *

{@code
- * Elasticsearch sink = Elasticsearch
- *     .builder()
- *     .setHosts(MY_ELASTICSEARCH_HOSTS)
- *     .setEmitter(MY_ELASTICSEARCH_EMITTER)
- *     .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
- *     .build();
- * }
+ * Base builder to construct a {@link ElasticsearchSink}. * * @param type of the records converted to Elasticsearch actions */ @PublicEvolving -public class ElasticsearchSinkBuilder { +public abstract class ElasticsearchSinkBuilderBase { private int bulkFlushMaxActions = 1000; private int bulkFlushMaxMb = -1; @@ -62,14 +51,19 @@ public class ElasticsearchSinkBuilder { private String password; private String connectionPathPrefix; - ElasticsearchSinkBuilder() {} + protected ElasticsearchSinkBuilderBase() {} + + @SuppressWarnings("unchecked") + private ElasticsearchSinkBuilderBase self() { + return (ElasticsearchSinkBuilderBase) this; + } /** * Sets the hosts where the Elasticsearch cluster nodes are reachable. * * @param hosts http addresses describing the node locations */ - public ElasticsearchSinkBuilder setHosts(HttpHost... hosts) { + public ElasticsearchSinkBuilderBase setHosts(HttpHost... hosts) { checkNotNull(hosts); checkState(hosts.length > 0, "Hosts cannot be empty."); this.hosts = Arrays.asList(hosts); @@ -80,15 +74,16 @@ public ElasticsearchSinkBuilder setHosts(HttpHost... hosts) { * Sets the emitter which is invoked on every record to convert it to Elasticsearch actions. * * @param emitter to process records into Elasticsearch actions. - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setEmitter( + public ElasticsearchSinkBuilderBase setEmitter( ElasticsearchEmitter emitter) { checkNotNull(emitter); checkState( InstantiationUtil.isSerializable(emitter), "The elasticsearch emitter must be serializable."); - final ElasticsearchSinkBuilder self = self(); + + final ElasticsearchSinkBuilderBase self = self(); self.emitter = emitter; return self; } @@ -98,9 +93,10 @@ public ElasticsearchSinkBuilder setEmitter( * DeliveryGuarantee#NONE} * * @param deliveryGuarantee which describes the record emission behaviour - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { + public ElasticsearchSinkBuilderBase setDeliveryGuarantee( + DeliveryGuarantee deliveryGuarantee) { checkState( deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, "Elasticsearch sink does not support the EXACTLY_ONCE guarantee."); @@ -113,9 +109,9 @@ public ElasticsearchSinkBuilder setDeliveryGuarantee(DeliveryGuarantee deliv * disable it. The default flush size 1000. * * @param numMaxActions the maximum number of actions to buffer per bulk request. - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setBulkFlushMaxActions(int numMaxActions) { + public ElasticsearchSinkBuilderBase setBulkFlushMaxActions(int numMaxActions) { checkState( numMaxActions == -1 || numMaxActions > 0, "Max number of buffered actions must be larger than 0."); @@ -128,9 +124,9 @@ public ElasticsearchSinkBuilder setBulkFlushMaxActions(int numMaxActions) { * disable it. * * @param maxSizeMb the maximum size of buffered actions, in mb. - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setBulkFlushMaxSizeMb(int maxSizeMb) { + public ElasticsearchSinkBuilderBase setBulkFlushMaxSizeMb(int maxSizeMb) { checkState( maxSizeMb == -1 || maxSizeMb > 0, "Max size of buffered actions must be larger than 0."); @@ -142,9 +138,9 @@ public ElasticsearchSinkBuilder setBulkFlushMaxSizeMb(int maxSizeMb) { * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it. * * @param intervalMillis the bulk flush interval, in milliseconds. - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setBulkFlushInterval(long intervalMillis) { + public ElasticsearchSinkBuilderBase setBulkFlushInterval(long intervalMillis) { checkState( intervalMillis == -1 || intervalMillis >= 0, "Interval (in milliseconds) between each flush must be larger than " @@ -163,9 +159,9 @@ public ElasticsearchSinkBuilder setBulkFlushInterval(long intervalMillis) { *

Sets the maximum number of retries for a backoff attempt when flushing bulk requests. * * @param flushBackoffType the backoff type to use. - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setBulkFlushBackoffStrategy( + public ElasticsearchSinkBuilderBase setBulkFlushBackoffStrategy( FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) { this.bulkFlushBackoffType = checkNotNull(flushBackoffType); checkState( @@ -185,9 +181,9 @@ public ElasticsearchSinkBuilder setBulkFlushBackoffStrategy( * Sets the username used to authenticate the connection with the Elasticsearch cluster. * * @param username of the Elasticsearch cluster user - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setConnectionUsername(String username) { + public ElasticsearchSinkBuilderBase setConnectionUsername(String username) { checkNotNull(username); this.username = username; return this; @@ -197,9 +193,9 @@ public ElasticsearchSinkBuilder setConnectionUsername(String username) { * Sets the password used to authenticate the conection with the Elasticsearch cluster. * * @param password of the Elasticsearch cluster user - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setConnectionPassword(String password) { + public ElasticsearchSinkBuilderBase setConnectionPassword(String password) { checkNotNull(password); this.password = password; return this; @@ -209,38 +205,49 @@ public ElasticsearchSinkBuilder setConnectionPassword(String password) { * Sets a prefix which used for every REST communication to the Elasticsearch cluster. * * @param prefix for the communication - * @return {@link ElasticsearchSinkBuilder} + * @return this builder */ - public ElasticsearchSinkBuilder setConnectionPathPrefix(String prefix) { + public ElasticsearchSinkBuilderBase setConnectionPathPrefix(String prefix) { checkNotNull(prefix); this.connectionPathPrefix = prefix; return this; } - /** @return {@link ElasticsearchSink} */ + abstract BulkRequestConsumerFactory getBulkRequestConsumer(); + + /** + * Constructs the {@link ElasticsearchSink} with the properties configured this builder. + * + * @return {@link ElasticsearchSink} + */ public ElasticsearchSink build() { - checkNotNull(hosts); checkNotNull(emitter); - return new ElasticsearchSink( - hosts, - emitter, - deliveryGuarantee, - buildBulkProcessorConfig(), - new NetworkClientConfig(username, password, connectionPathPrefix)); + checkNotNull(hosts); + + NetworkClientConfig networkClientConfig = buildNetworkClientConfig(); + BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig(); + + return new ElasticsearchSink<>( + hosts, emitter, deliveryGuarantee, bulkProcessorConfig, networkClientConfig); } - @SuppressWarnings("unchecked") - private ElasticsearchSinkBuilder self() { - return (ElasticsearchSinkBuilder) this; + private NetworkClientConfig buildNetworkClientConfig() { + checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); + + return new NetworkClientConfig(username, password, connectionPathPrefix); } private BulkProcessorConfig buildBulkProcessorConfig() { + BulkRequestConsumerFactory bulkRequestConsumer = getBulkRequestConsumer(); + checkNotNull(bulkRequestConsumer); + return new BulkProcessorConfig( bulkFlushMaxActions, bulkFlushMaxMb, bulkFlushInterval, bulkFlushBackoffType, bulkFlushBackoffRetries, - bulkFlushBackOffDelay); + bulkFlushBackOffDelay, + bulkRequestConsumer); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java similarity index 96% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index d47fb7f9..59a4fc85 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -39,7 +39,6 @@ import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; @@ -89,7 +88,7 @@ class ElasticsearchWriter implements SinkWriter { * @param metricGroup for the sink writer * @param mailboxExecutor Flink's mailbox executor */ - public ElasticsearchWriter( + ElasticsearchWriter( List hosts, ElasticsearchEmitter emitter, boolean flushOnCheckpoint, @@ -108,11 +107,7 @@ public ElasticsearchWriter( this.bulkProcessor = configureBulkProcessor( BulkProcessor.builder( - (bulkRequest, bulkResponseActionListener) -> - client.bulkAsync( - bulkRequest, - RequestOptions.DEFAULT, - bulkResponseActionListener), + bulkProcessorConfig.getBulkRequestConsumerFactory().create(client), new BulkListener()), bulkProcessorConfig); this.requestIndexer = new DefaultRequestIndexer(); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java similarity index 81% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java index 3c0a62a4..b1fbd8c1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java @@ -23,12 +23,11 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.DockerImageVersions; -import org.apache.flink.util.TestLogger; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; @@ -42,16 +41,12 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.junit.jupiter.Container; -import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; import javax.annotation.Nullable; @@ -65,40 +60,41 @@ import static org.junit.jupiter.api.Assertions.assertTrue; /** Tests for {@link ElasticsearchSink}. */ -@Testcontainers -class ElasticsearchSinkITCase extends TestLogger { - - private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); - private static final String ELASTICSEARCH_PASSWORD = "test-password"; - private static final String ELASTICSEARCH_USER = "elastic"; - - @Container - private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer( - DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_7)) - .withPassword(ELASTICSEARCH_PASSWORD) - .withLogConsumer(new Slf4jLogConsumer(LOG)); +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchSinkBaseITCase { + protected static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkBaseITCase.class); + protected static final String ELASTICSEARCH_PASSWORD = "test-password"; + protected static final String ELASTICSEARCH_USER = "elastic"; private static boolean failed; private RestHighLevelClient client; - private TestClient context; + private TestClientBase context; - @BeforeEach - void setUp() { - failed = false; + abstract String getElasticsearchHttpHostAddress(); + + abstract TestClientBase createTestClient(RestHighLevelClient client); + + abstract ElasticsearchSinkBuilderBase> getSinkBuilder(); + + private RestHighLevelClient createRestHighLevelClient() { final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); credentialsProvider.setCredentials( AuthScope.ANY, new UsernamePasswordCredentials(ELASTICSEARCH_USER, ELASTICSEARCH_PASSWORD)); - client = - new RestHighLevelClient( - RestClient.builder(HttpHost.create(ES_CONTAINER.getHttpHostAddress())) - .setHttpClientConfigCallback( - httpClientBuilder -> - httpClientBuilder.setDefaultCredentialsProvider( - credentialsProvider))); - context = new TestClient(client); + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(getElasticsearchHttpHostAddress())) + .setHttpClientConfigCallback( + httpClientBuilder -> + httpClientBuilder.setDefaultCredentialsProvider( + credentialsProvider))); + } + + @BeforeEach + void setUp() { + failed = false; + client = createRestHighLevelClient(); + context = createTestClient(client); } @AfterEach @@ -165,8 +161,8 @@ private void runTest( @Nullable MapFunction additionalMapper) throws Exception { final ElasticsearchSink> sink = - ElasticsearchSink.builder() - .setHosts(HttpHost.create(ES_CONTAINER.getHttpHostAddress())) + getSinkBuilder() + .setHosts(HttpHost.create(getElasticsearchHttpHostAddress())) .setEmitter(emitterProvider.apply(index, context.getDataFieldName())) .setBulkFlushMaxActions(5) .setConnectionUsername(ELASTICSEARCH_USER) @@ -193,7 +189,7 @@ private void runTest( public Tuple2 map(Long value) throws Exception { return Tuple2.of( value.intValue(), - TestClient.buildMessage(value.intValue())); + TestClientBase.buildMessage(value.intValue())); } }) .sinkTo(sink); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java similarity index 59% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index b72411cb..270b15b1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -18,12 +18,14 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.util.TestLogger; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; import org.apache.http.HttpHost; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -31,12 +33,14 @@ import static org.junit.jupiter.api.Assertions.assertThrows; -/** Tests for {@link ElasticsearchSinkBuilder}. */ -class ElasticsearchSinkBuilderTest extends TestLogger { +/** Tests for {@link ElasticsearchSinkBuilderBase}. */ +@ExtendWith(TestLoggerExtension.class) +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +abstract class ElasticsearchSinkBuilderBaseTest { @ParameterizedTest @MethodSource("validBuilders") - void testBuildElasticsearchSink(ElasticsearchSinkBuilder builder) { + void testBuildElasticsearchSink(ElasticsearchSinkBuilderBase builder) { builder.build(); } @@ -44,42 +48,34 @@ void testBuildElasticsearchSink(ElasticsearchSinkBuilder builder) { void testThrowIfExactlyOnceConfigured() { assertThrows( IllegalStateException.class, - () -> createBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); + () -> createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE)); } @Test void testThrowIfHostsNotSet() { assertThrows( NullPointerException.class, - () -> - new ElasticsearchSinkBuilder<>() - .setEmitter((element, indexer, context) -> {}) - .build()); + () -> createEmptyBuilder().setEmitter((element, indexer, context) -> {}).build()); } @Test void testThrowIfEmitterNotSet() { assertThrows( NullPointerException.class, - () -> - new ElasticsearchSinkBuilder<>() - .setHosts(new HttpHost("localhost:3000")) - .build()); + () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build()); } - private static List> validBuilders() { + private List> validBuilders() { return Lists.newArrayList( - createBuilder(), - createBuilder().setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), - createBuilder().setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), - createBuilder() + createMinimalBuilder(), + createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), + createMinimalBuilder().setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), + createMinimalBuilder() .setConnectionUsername("username") .setConnectionPassword("password")); } - private static ElasticsearchSinkBuilder createBuilder() { - return new ElasticsearchSinkBuilder<>() - .setEmitter((element, indexer, context) -> {}) - .setHosts(new HttpHost("localhost:3000")); - } + abstract ElasticsearchSinkBuilderBase createEmptyBuilder(); + + abstract ElasticsearchSinkBuilderBase createMinimalBuilder(); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java similarity index 80% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 9ea14db8..2d98157f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; @@ -28,15 +29,18 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.ThrowingRunnable; import org.apache.http.HttpHost; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -49,7 +53,7 @@ import java.util.Collections; import java.util.Optional; -import static org.apache.flink.connector.elasticsearch.sink.TestClient.buildMessage; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.buildMessage; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -57,7 +61,8 @@ /** Tests for {@link ElasticsearchWriter}. */ @Testcontainers -class ElasticsearchWriterITCase extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +class ElasticsearchWriterITCase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriterITCase.class); @@ -67,9 +72,17 @@ class ElasticsearchWriterITCase extends TestLogger { .withLogConsumer(new Slf4jLogConsumer(LOG)); private RestHighLevelClient client; - private TestClient context; + private TestClientBase context; private MetricListener metricListener; + private final BulkRequestConsumerFactory bulkRequestConsumerFactory = + (client) -> + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + @BeforeEach void setUp() { metricListener = new MetricListener(); @@ -91,7 +104,14 @@ void testWriteOnBulkFlush() throws Exception { final String index = "test-bulk-flush-without-checkpoint"; final int flushAfterNActions = 5; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + new BulkProcessorConfig( + flushAfterNActions, + -1, + -1, + FlushBackoffType.NONE, + 0, + 0, + bulkRequestConsumerFactory); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -124,7 +144,8 @@ void testWriteOnBulkIntervalFlush() throws Exception { // Configure bulk processor to flush every 1s; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0); + new BulkProcessorConfig( + -1, -1, 1000, FlushBackoffType.NONE, 0, 0, bulkRequestConsumerFactory); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -142,7 +163,8 @@ void testWriteOnBulkIntervalFlush() throws Exception { void testWriteOnCheckpoint() throws Exception { final String index = "test-bulk-flush-with-checkpoint"; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0); + new BulkProcessorConfig( + -1, -1, -1, FlushBackoffType.NONE, 0, 0, bulkRequestConsumerFactory); // Enable flush on checkpoint try (final ElasticsearchWriter> writer = @@ -170,7 +192,14 @@ void testIncrementByteOutMetric() throws Exception { metricListener.getMetricGroup(), operatorIOMetricGroup); final int flushAfterNActions = 2; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + new BulkProcessorConfig( + flushAfterNActions, + -1, + -1, + FlushBackoffType.NONE, + 0, + 0, + bulkRequestConsumerFactory); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig, metricGroup)) { @@ -197,7 +226,14 @@ void testCurrentSendTime() throws Exception { final String index = "test-current-send-time"; final int flushAfterNActions = 2; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + new BulkProcessorConfig( + flushAfterNActions, + -1, + -1, + FlushBackoffType.NONE, + 0, + 0, + bulkRequestConsumerFactory); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -227,7 +263,7 @@ private ElasticsearchWriter> createWriter( boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig, SinkWriterMetricGroup metricGroup) { - return new ElasticsearchWriter<>( + return new ElasticsearchWriter>( Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), TestEmitter.jsonEmitter(index, context.getDataFieldName()), flushOnCheckpoint, @@ -237,6 +273,18 @@ private ElasticsearchWriter> createWriter( new TestMailbox()); } + private static class TestClient extends TestClientBase { + + TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT); + } + } + private static class TestMailbox implements MailboxExecutor { @Override diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java similarity index 78% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java index 140d26b8..5d2f6f53 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClient.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java @@ -18,9 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; import java.io.IOException; @@ -28,22 +26,22 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -class TestClient { +abstract class TestClientBase { + static final String DOCUMENT_TYPE = "test-document-type"; private static final String DATA_FIELD_NAME = "data"; - private final RestHighLevelClient client; + final RestHighLevelClient client; - TestClient(RestHighLevelClient client) { + TestClientBase(RestHighLevelClient client) { this.client = client; } + abstract GetResponse getResponse(String index, int id) throws IOException; + void assertThatIdsAreNotWritten(String index, int... ids) throws IOException { for (final int id : ids) { try { - final GetResponse response = - client.get( - new GetRequest(index, Integer.toString(id)), - RequestOptions.DEFAULT); + final GetResponse response = getResponse(index, id); assertFalse( response.isExists(), String.format("Id %s is unexpectedly present.", id)); } catch (ElasticsearchStatusException e) { @@ -57,10 +55,7 @@ void assertThatIdsAreWritten(String index, int... ids) for (final int id : ids) { GetResponse response; do { - response = - client.get( - new GetRequest(index, Integer.toString(id)), - RequestOptions.DEFAULT); + response = getResponse(index, id); Thread.sleep(10); } while (response.isSourceEmpty()); assertEquals(buildMessage(id), response.getSource().get(DATA_FIELD_NAME)); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java index 41d3a5eb..73b512d7 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -29,6 +29,8 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; + class TestEmitter implements ElasticsearchEmitter> { private final String index; @@ -62,6 +64,7 @@ public IndexRequest createIndexRequest(Tuple2 element) { try { return new IndexRequest(index) .id(element.f0.toString()) + .type(DOCUMENT_TYPE) .source(xContentBuilderProvider.getBuilder().map(document)); } catch (IOException e) { throw new RuntimeException(e); diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index a5e25f76..dd147d9f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -131,6 +131,13 @@ under the License. test + + org.apache.flink + flink-connector-testing + ${project.version} + test + + diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java new file mode 100644 index 00000000..e766cb11 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -0,0 +1,54 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * Builder to construct an Elasticsearch 6 compatible {@link ElasticsearchSink}. + * + *

The following example shows the minimal setup to create a ElasticsearchSink that submits + * actions on checkpoint or the default number of actions was buffered (1000). + * + *

{@code
+ * ElasticsearchSink sink = new Elasticsearch6SinkBuilder()
+ *     .setHosts(new HttpHost("localhost:9200")
+ *     .setEmitter((element, context, indexer) -> {
+ *          indexer.add(
+ *              new IndexRequest("my-index","my-type")
+ *              .id(element.f0.toString())
+ *              .source(element.f1)
+ *          );
+ *      })
+ *     .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+ *     .build();
+ * }
+ * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class Elasticsearch6SinkBuilder extends ElasticsearchSinkBuilderBase { + + public Elasticsearch6SinkBuilder() {} + + @Override + public BulkRequestConsumerFactory getBulkRequestConsumer() { + return client -> client::bulkAsync; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java new file mode 100644 index 00000000..0372caa0 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java @@ -0,0 +1,36 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.http.HttpHost; + +/** Tests for {@link Elasticsearch6SinkBuilder}. */ +class Elasticsearch6SinkBuilderTest extends ElasticsearchSinkBuilderBaseTest { + + @Override + ElasticsearchSinkBuilderBase createEmptyBuilder() { + return new Elasticsearch6SinkBuilder<>(); + } + + @Override + ElasticsearchSinkBuilderBase createMinimalBuilder() { + return new Elasticsearch6SinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java new file mode 100644 index 00000000..3df6c9af --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java @@ -0,0 +1,55 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.client.RestHighLevelClient; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class Elasticsearch6SinkITCase extends ElasticsearchSinkBaseITCase { + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer( + DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_6)) + .withPassword(ELASTICSEARCH_PASSWORD) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + TestClientBase createTestClient(RestHighLevelClient client) { + return new Elasticsearch6TestClient(client); + } + + @Override + ElasticsearchSinkBuilderBase> getSinkBuilder() { + return new Elasticsearch6SinkBuilder<>(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java new file mode 100644 index 00000000..a29eb03a --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java @@ -0,0 +1,36 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +class Elasticsearch6TestClient extends TestClientBase { + + Elasticsearch6TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, DOCUMENT_TYPE, Integer.toString(id))); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index bfee4f89..db0e72d8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -121,6 +121,13 @@ under the License. test
+ + org.apache.flink + flink-connector-testing + ${project.version} + test + + diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java new file mode 100644 index 00000000..30986dc0 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -0,0 +1,58 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.PublicEvolving; + +import org.elasticsearch.client.RequestOptions; + +/** + * Builder to construct an Elasticsearch 7 compatible {@link ElasticsearchSink}. + * + *

The following example shows the minimal setup to create a ElasticsearchSink that submits + * actions on checkpoint or the default number of actions was buffered (1000). + * + *

{@code
+ * ElasticsearchSink sink = new Elasticsearch7SinkBuilder()
+ *     .setHosts(new HttpHost("localhost:9200")
+ *     .setEmitter((element, context, indexer) -> {
+ *          indexer.add(
+ *              new IndexRequest("my-index")
+ *              .id(element.f0.toString())
+ *              .source(element.f1)
+ *          );
+ *      })
+ *     .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+ *     .build();
+ * }
+ * + * @param type of the records converted to Elasticsearch actions + */ +@PublicEvolving +public class Elasticsearch7SinkBuilder extends ElasticsearchSinkBuilderBase { + + public Elasticsearch7SinkBuilder() {} + + @Override + public BulkRequestConsumerFactory getBulkRequestConsumer() { + return (client) -> + (bulkRequest, bulkResponseActionListener) -> + client.bulkAsync( + bulkRequest, RequestOptions.DEFAULT, bulkResponseActionListener); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java index e6a03dc3..584fad8a 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -20,8 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder; import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink; -import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilder; import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.EncodingFormat; @@ -90,7 +90,7 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { KeyExtractor.createKeyExtractor( primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter())); - final ElasticsearchSinkBuilder builder = ElasticsearchSink.builder(); + final Elasticsearch7SinkBuilder builder = new Elasticsearch7SinkBuilder<>(); builder.setEmitter(rowElasticsearchEmitter); builder.setHosts(config.getHosts().toArray(new HttpHost[0])); builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java new file mode 100644 index 00000000..052dd8ad --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java @@ -0,0 +1,36 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.http.HttpHost; + +/** Tests for {@link Elasticsearch7SinkBuilder}. */ +class Elasticsearch7SinkBuilderTest extends ElasticsearchSinkBuilderBaseTest { + + @Override + ElasticsearchSinkBuilderBase createEmptyBuilder() { + return new Elasticsearch7SinkBuilder<>(); + } + + @Override + ElasticsearchSinkBuilderBase createMinimalBuilder() { + return new Elasticsearch7SinkBuilder<>() + .setEmitter((element, indexer, context) -> {}) + .setHosts(new HttpHost("localhost:3000")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java new file mode 100644 index 00000000..25596df7 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java @@ -0,0 +1,55 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.util.DockerImageVersions; + +import org.elasticsearch.client.RestHighLevelClient; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; +import org.testcontainers.utility.DockerImageName; + +/** Tests for {@link ElasticsearchSink}. */ +@Testcontainers +class Elasticsearch7SinkITCase extends ElasticsearchSinkBaseITCase { + + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer( + DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_7)) + .withPassword(ELASTICSEARCH_PASSWORD) + .withLogConsumer(new Slf4jLogConsumer(LOG)); + + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); + } + + @Override + TestClientBase createTestClient(RestHighLevelClient client) { + return new Elasticsearch7TestClient(client); + } + + @Override + ElasticsearchSinkBuilderBase> getSinkBuilder() { + return new Elasticsearch7SinkBuilder<>(); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java new file mode 100644 index 00000000..969c1796 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java @@ -0,0 +1,37 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.IOException; + +class Elasticsearch7TestClient extends TestClientBase { + + Elasticsearch7TestClient(RestHighLevelClient client) { + super(client); + } + + @Override + GetResponse getResponse(String index, int id) throws IOException { + return client.get(new GetRequest(index, Integer.toString(id)), RequestOptions.DEFAULT); + } +} From b187b8d14e41893952896992f4237d531a1cfb0d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 21 Oct 2021 13:14:19 +0200 Subject: [PATCH 163/207] [FLINK-24325][connectors/elasticsearch] Integrate common unified Elasticsearch sink with Table API for versions 6 and 7 --- .../table/ElasticsearchConfiguration.java | 155 +++---- .../table/ElasticsearchConnectorOptions.java | 80 +--- .../table/ElasticsearchDynamicSink.java} | 84 +++- .../ElasticsearchDynamicSinkFactoryBase.java | 222 ++++++++++ .../ElasticsearchSinkBuilderSupplier.java | 25 ++ .../table/ElasticsearchValidationUtils.java | 56 --- .../elasticsearch/table/RequestFactory.java | 54 --- .../table/RowElasticsearchEmitter.java | 12 +- .../table/RowElasticsearchSinkFunction.java | 140 ------ .../ElasticsearchDynamicSinkBaseITCase.java | 327 ++++++++++++++ ...asticsearchDynamicSinkFactoryBaseTest.java | 230 ++++++++++ .../table/Elasticsearch6Configuration.java | 52 +-- .../table/Elasticsearch6ConnectorOptions.java | 39 ++ .../table/Elasticsearch6DynamicSink.java | 317 -------------- .../Elasticsearch6DynamicSinkFactory.java | 152 ++----- .../Elasticsearch6DynamicSinkFactoryTest.java | 233 +--------- .../Elasticsearch6DynamicSinkITCase.java | 405 +++--------------- .../table/Elasticsearch6DynamicSinkTest.java | 294 ------------- .../table/Elasticsearch7Configuration.java | 137 ------ .../table/Elasticsearch7ConnectorOptions.java | 124 ------ .../Elasticsearch7DynamicSinkFactory.java | 171 +------- .../Elasticsearch7DynamicSinkFactoryTest.java | 215 +--------- .../Elasticsearch7DynamicSinkITCase.java | 357 ++------------- 23 files changed, 1164 insertions(+), 2717 deletions(-) rename flink-connectors/{flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java => flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java} (66%) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java rename flink-connectors/{flink-connector-elasticsearch7 => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java (92%) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java delete mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java index 04c76333..78a31022 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -19,87 +19,57 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.util.InstantiationUtil; + +import org.apache.http.HttpHost; import java.time.Duration; -import java.util.Objects; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.util.Preconditions.checkNotNull; -/** Accessor methods to elasticsearch options. */ +/** Elasticsearch base configuration. */ @Internal class ElasticsearchConfiguration { protected final ReadableConfig config; - private final ClassLoader classLoader; - ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { - this.config = config; - this.classLoader = classLoader; - } - - public ActionRequestFailureHandler getFailureHandler() { - final ActionRequestFailureHandler failureHandler; - String value = config.get(FAILURE_HANDLER_OPTION); - switch (value.toUpperCase()) { - case "FAIL": - failureHandler = new NoOpFailureHandler(); - break; - case "IGNORE": - failureHandler = new IgnoringFailureHandler(); - break; - case "RETRY-REJECTED": - failureHandler = new RetryRejectedExecutionFailureHandler(); - break; - default: - try { - Class failureHandlerClass = Class.forName(value, false, classLoader); - failureHandler = - (ActionRequestFailureHandler) - InstantiationUtil.instantiate(failureHandlerClass); - } catch (ClassNotFoundException e) { - throw new ValidationException( - "Could not instantiate the failure handler class: " + value, e); - } - break; - } - return failureHandler; - } - - public String getDocumentType() { - return config.get(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION); + ElasticsearchConfiguration(ReadableConfig config) { + this.config = checkNotNull(config); } public int getBulkFlushMaxActions() { - int maxActions = config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return maxActions == 0 ? -1 : maxActions; + return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION); } - public long getBulkFlushMaxByteSize() { - long maxSize = - config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return maxSize == 0 ? -1 : maxSize; + public MemorySize getBulkFlushMaxByteSize() { + return config.get(BULK_FLUSH_MAX_SIZE_OPTION); } public long getBulkFlushInterval() { - long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); - // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. - return interval == 0 ? -1 : interval; + return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + } + + public DeliveryGuarantee getDeliveryGuarantee() { + return config.get(DELIVERY_GUARANTEE_OPTION); } public Optional getUsername() { @@ -110,20 +80,8 @@ public Optional getPassword() { return config.getOptional(PASSWORD_OPTION); } - public boolean isBulkFlushBackoffEnabled() { - return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) - != ElasticsearchConnectorOptions.BackOffType.DISABLED; - } - - public Optional getBulkFlushBackoffType() { - switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { - case CONSTANT: - return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); - case EXPONENTIAL: - return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - default: - return Optional.empty(); - } + public Optional getBulkFlushBackoffType() { + return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); } public Optional getBulkFlushBackoffRetries() { @@ -134,36 +92,47 @@ public Optional getBulkFlushBackoffDelay() { return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); } - public boolean isDisableFlushOnCheckpoint() { - return !config.get(ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION); - } - public String getIndex() { - return config.get(ElasticsearchConnectorOptions.INDEX_OPTION); + return config.get(INDEX_OPTION); } public String getKeyDelimiter() { - return config.get(ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION); + return config.get(KEY_DELIMITER_OPTION); } public Optional getPathPrefix() { - return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; + return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(ElasticsearchConfiguration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); } - ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; - return Objects.equals(config, that.config) && Objects.equals(classLoader, that.classLoader); - } - - @Override - public int hashCode() { - return Objects.hash(config, classLoader); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 1b0fc5e4..404d3ca1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -22,18 +22,21 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.description.Description; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import java.time.Duration; import java.util.List; -import static org.apache.flink.configuration.description.TextElement.text; - -/** Options for the Elasticsearch connector. */ +/** + * Base options for the Elasticsearch connector. Needs to be public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ @PublicEvolving public class ElasticsearchConnectorOptions { + ElasticsearchConnectorOptions() {} + public static final ConfigOption> HOSTS_OPTION = ConfigOptions.key("hosts") .stringType() @@ -47,12 +50,6 @@ public class ElasticsearchConnectorOptions { .noDefaultValue() .withDescription("Elasticsearch index for every record."); - public static final ConfigOption DOCUMENT_TYPE_OPTION = - ConfigOptions.key("document-type") - .stringType() - .noDefaultValue() - .withDescription("Elasticsearch document type."); - public static final ConfigOption PASSWORD_OPTION = ConfigOptions.key("password") .stringType() @@ -72,38 +69,13 @@ public class ElasticsearchConnectorOptions { .withDescription( "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); - public static final ConfigOption FAILURE_HANDLER_OPTION = - ConfigOptions.key("failure-handler") - .stringType() - .defaultValue("fail") - .withDescription( - Description.builder() - .text( - "Failure handling strategy in case a request to Elasticsearch fails") - .list( - text( - "\"fail\" (throws an exception if a request fails and thus causes a job failure)"), - text( - "\"ignore\" (ignores failures and drops the request)"), - text( - "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation)"), - text( - "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) - .build()); - - public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = - ConfigOptions.key("sink.flush-on-checkpoint") - .booleanType() - .defaultValue(true) - .withDescription("Disables flushing on checkpoint"); - public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = ConfigOptions.key("sink.bulk-flush.max-actions") .intType() .defaultValue(1000) .withDescription("Maximum number of actions to buffer for each bulk request."); - public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + public static final ConfigOption BULK_FLUSH_MAX_SIZE_OPTION = ConfigOptions.key("sink.bulk-flush.max-size") .memoryType() .defaultValue(MemorySize.parse("2mb")) @@ -115,10 +87,10 @@ public class ElasticsearchConnectorOptions { .defaultValue(Duration.ofSeconds(1)) .withDescription("Bulk flush interval"); - public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = ConfigOptions.key("sink.bulk-flush.backoff.strategy") - .enumType(BackOffType.class) - .defaultValue(BackOffType.DISABLED) + .enumType(FlushBackoffType.class) + .noDefaultValue() .withDescription("Backoff strategy"); public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = @@ -133,13 +105,7 @@ public class ElasticsearchConnectorOptions { .noDefaultValue() .withDescription("Delay between each backoff attempt."); - public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = - ConfigOptions.key("connection.max-retry-timeout") - .durationType() - .noDefaultValue() - .withDescription("Maximum timeout between retries."); - - public static final ConfigOption CONNECTION_PATH_PREFIX = + public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION = ConfigOptions.key("connection.path-prefix") .stringType() .noDefaultValue() @@ -153,19 +119,9 @@ public class ElasticsearchConnectorOptions { "The format must produce a valid JSON document. " + "Please refer to the documentation on formats for more details."); - // -------------------------------------------------------------------------------------------- - // Enums - // -------------------------------------------------------------------------------------------- - - /** - * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code - * DISABLED} option. - */ - public enum BackOffType { - DISABLED, - CONSTANT, - EXPONENTIAL - } - - private ElasticsearchConnectorOptions() {} + public static final ConfigOption DELIVERY_GUARANTEE_OPTION = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.NONE) + .withDescription("Optional delivery guarantee when committing."); } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java similarity index 66% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java index 584fad8a..3c2b6bd3 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java @@ -20,8 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder; import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink; +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.EncodingFormat; @@ -35,8 +35,11 @@ import org.apache.http.HttpHost; import org.elasticsearch.common.xcontent.XContentType; +import javax.annotation.Nullable; + import java.util.List; import java.util.Objects; +import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -45,22 +48,44 @@ * logical description. */ @Internal -final class Elasticsearch7DynamicSink implements DynamicTableSink { +class ElasticsearchDynamicSink implements DynamicTableSink { + + final EncodingFormat> format; + final DataType physicalRowDataType; + final List primaryKeyLogicalTypesWithIndex; + final ElasticsearchConfiguration config; - private final EncodingFormat> format; - private final DataType physicalRowDataType; - private final List primaryKeyLogicalTypesWithIndex; - private final Elasticsearch7Configuration config; + final String summaryString; + final ElasticsearchSinkBuilderSupplier builderSupplier; + @Nullable final String documentType; - Elasticsearch7DynamicSink( + ElasticsearchDynamicSink( EncodingFormat> format, - Elasticsearch7Configuration config, + ElasticsearchConfiguration config, List primaryKeyLogicalTypesWithIndex, - DataType physicalRowDataType) { + DataType physicalRowDataType, + String summaryString, + ElasticsearchSinkBuilderSupplier builderSupplier, + @Nullable String documentType) { this.format = checkNotNull(format); this.physicalRowDataType = checkNotNull(physicalRowDataType); this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex); this.config = checkNotNull(config); + this.summaryString = checkNotNull(summaryString); + this.builderSupplier = checkNotNull(builderSupplier); + this.documentType = documentType; + } + + Function createKeyExtractor() { + return KeyExtractor.createKeyExtractor( + primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter()); + } + + IndexGenerator createIndexGenerator() { + return IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), + DataType.getFieldNames(physicalRowDataType), + DataType.getFieldDataTypes(physicalRowDataType)); } @Override @@ -81,21 +106,18 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { final RowElasticsearchEmitter rowElasticsearchEmitter = new RowElasticsearchEmitter( - IndexGeneratorFactory.createIndexGenerator( - config.getIndex(), - DataType.getFieldNames(physicalRowDataType), - DataType.getFieldDataTypes(physicalRowDataType)), + createIndexGenerator(), format, XContentType.JSON, - KeyExtractor.createKeyExtractor( - primaryKeyLogicalTypesWithIndex, config.getKeyDelimiter())); + documentType, + createKeyExtractor()); - final Elasticsearch7SinkBuilder builder = new Elasticsearch7SinkBuilder<>(); + final ElasticsearchSinkBuilderBase builder = builderSupplier.get(); builder.setEmitter(rowElasticsearchEmitter); builder.setHosts(config.getHosts().toArray(new HttpHost[0])); builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); - builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushMaxSizeMb(config.getBulkFlushMaxByteSize().getMebiBytes()); builder.setBulkFlushInterval(config.getBulkFlushInterval()); if (config.getBulkFlushBackoffType().isPresent()) { @@ -126,13 +148,19 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { @Override public DynamicTableSink copy() { - return new Elasticsearch7DynamicSink( - format, config, primaryKeyLogicalTypesWithIndex, physicalRowDataType); + return new ElasticsearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + physicalRowDataType, + summaryString, + builderSupplier, + documentType); } @Override public String asSummaryString() { - return "Elasticsearch7"; + return summaryString; } @Override @@ -143,16 +171,26 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + ElasticsearchDynamicSink that = (ElasticsearchDynamicSink) o; return Objects.equals(format, that.format) && Objects.equals(physicalRowDataType, that.physicalRowDataType) && Objects.equals( primaryKeyLogicalTypesWithIndex, that.primaryKeyLogicalTypesWithIndex) - && Objects.equals(config, that.config); + && Objects.equals(config, that.config) + && Objects.equals(summaryString, that.summaryString) + && Objects.equals(builderSupplier, that.builderSupplier) + && Objects.equals(documentType, that.documentType); } @Override public int hashCode() { - return Objects.hash(format, physicalRowDataType, primaryKeyLogicalTypesWithIndex, config); + return Objects.hash( + format, + physicalRowDataType, + primaryKeyLogicalTypesWithIndex, + config, + summaryString, + builderSupplier, + documentType); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java new file mode 100644 index 00000000..6d545289 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -0,0 +1,222 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.StringUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.commons.lang3.StringUtils.capitalize; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link DynamicTableSinkFactory} for discovering ElasticsearchDynamicSink. */ +@Internal +abstract class ElasticsearchDynamicSinkFactoryBase implements DynamicTableSinkFactory { + + private final String factoryIdentifier; + private final ElasticsearchSinkBuilderSupplier sinkBuilderSupplier; + + public ElasticsearchDynamicSinkFactoryBase( + String factoryIdentifier, + ElasticsearchSinkBuilderSupplier sinkBuilderSupplier) { + this.factoryIdentifier = checkNotNull(factoryIdentifier); + this.sinkBuilderSupplier = checkNotNull(sinkBuilderSupplier); + } + + @Nullable + String getDocumentType(Context context) { + return null; // document type is only set in Elasticsearch versions < 7 + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + List primaryKeyLogicalTypesWithIndex = + getPrimaryKeyLogicalTypesWithIndex(context); + EncodingFormat> format = + getValidatedEncodingFormat(this, context); + + ElasticsearchConfiguration config = getConfiguration(context); + validateConfiguration(config); + + return new ElasticsearchDynamicSink( + format, + config, + primaryKeyLogicalTypesWithIndex, + context.getPhysicalRowDataType(), + capitalize(factoryIdentifier), + sinkBuilderSupplier, + getDocumentType(context)); + } + + ElasticsearchConfiguration getConfiguration(Context context) { + return new ElasticsearchConfiguration( + Configuration.fromMap(context.getCatalogTable().getOptions())); + } + + void validateConfiguration(ElasticsearchConfiguration config) { + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize().getBytes(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLUSH_MAX_SIZE_OPTION.key(), + config.getBulkFlushMaxByteSize().toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + EncodingFormat> getValidatedEncodingFormat( + DynamicTableFactory factory, DynamicTableFactory.Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(factory, context); + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + helper.validate(); + return format; + } + + List getPrimaryKeyLogicalTypesWithIndex(Context context) { + DataType physicalRowDataType = context.getPhysicalRowDataType(); + int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); + if (primaryKeyIndexes.length != 0) { + DataType pkDataType = DataType.projectFields(physicalRowDataType, primaryKeyIndexes); + + ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); + } + + ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); + return Arrays.stream(primaryKeyIndexes) + .mapToObj( + index -> { + Optional column = resolvedSchema.getColumn(index); + if (!column.isPresent()) { + throw new IllegalStateException( + String.format( + "No primary key column found with index '%s'.", + index)); + } + LogicalType logicalType = column.get().getDataType().getLogicalType(); + return new LogicalTypeWithIndex(index, logicalType); + }) + .collect(Collectors.toList()); + } + + @Override + public Set> requiredOptions() { + return Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + } + + @Override + public Set> optionalOptions() { + return Stream.of( + KEY_DELIMITER_OPTION, + BULK_FLUSH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX_OPTION, + FORMAT_OPTION, + DELIVERY_GUARANTEE_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + } + + @Override + public String factoryIdentifier() { + return factoryIdentifier; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java new file mode 100644 index 00000000..1c3ce9fa --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java @@ -0,0 +1,25 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; + +import java.util.function.Supplier; + +interface ElasticsearchSinkBuilderSupplier extends Supplier> {} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java index b2ea6ffe..40d89196 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -19,11 +19,9 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -35,20 +33,9 @@ /** Utility methods for validating Elasticsearch properties. */ @Internal class ElasticsearchValidationUtils { - - private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); private static final Set ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); static { - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); - ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); - ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR); ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR); ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN); @@ -68,49 +55,6 @@ class ElasticsearchValidationUtils { ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME); } - /** - * Checks that the table does not have primary key defined on illegal types. In Elasticsearch - * the primary key is used to calculate the Elasticsearch document id, which is a string of up - * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the - * fields. Certain types do not have a good string representation to be used in this scenario. - * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link - * LogicalTypeRoot#RAW} type. - */ - @Deprecated - public static void validatePrimaryKey(TableSchema schema) { - schema.getPrimaryKey() - .ifPresent( - key -> { - List illegalTypes = - key.getColumns().stream() - .map( - fieldName -> { - LogicalType logicalType = - schema.getFieldDataType(fieldName) - .get() - .getLogicalType(); - if (logicalType.is( - LogicalTypeRoot.DISTINCT_TYPE)) { - return ((DistinctType) logicalType) - .getSourceType() - .getTypeRoot(); - } else { - return logicalType.getTypeRoot(); - } - }) - .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) - .collect(Collectors.toList()); - - if (!illegalTypes.isEmpty()) { - throw new ValidationException( - String.format( - "The table has a primary key on columns of illegal types: %s.\n" - + " Elasticsearch sink does not support primary keys on columns of types: %s.", - illegalTypes, ILLEGAL_PRIMARY_KEY_TYPES)); - } - }); - } - /** * Checks that the table does not have a primary key defined on illegal types. In Elasticsearch * the primary key is used to calculate the Elasticsearch document id, which is a string of up diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java deleted file mode 100644 index f5b24180..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.common.xcontent.XContentType; - -import java.io.Serializable; - -/** For version-agnostic creating of {@link ActionRequest}s. */ -@Internal -interface RequestFactory extends Serializable { - /** - * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field has - * been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - UpdateRequest createUpdateRequest( - String index, String docType, String key, XContentType contentType, byte[] document); - - /** - * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field has - * been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - IndexRequest createIndexRequest( - String index, String docType, String key, XContentType contentType, byte[] document); - - /** - * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field has - * been deprecated since Elasticsearch 7.x and it would not take any effort. - */ - DeleteRequest createDeleteRequest(String index, String docType, String key); -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java similarity index 92% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java index 8c1f2081..236791c7 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java @@ -31,6 +31,8 @@ import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.common.xcontent.XContentType; +import javax.annotation.Nullable; + import java.util.function.Function; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -41,16 +43,19 @@ class RowElasticsearchEmitter implements ElasticsearchEmitter { private final IndexGenerator indexGenerator; private final SerializationSchema serializationSchema; private final XContentType contentType; + @Nullable private final String documentType; private final Function createKey; public RowElasticsearchEmitter( IndexGenerator indexGenerator, SerializationSchema serializationSchema, XContentType contentType, + @Nullable String documentType, Function createKey) { this.indexGenerator = checkNotNull(indexGenerator); this.serializationSchema = checkNotNull(serializationSchema); this.contentType = checkNotNull(contentType); + this.documentType = documentType; this.createKey = checkNotNull(createKey); } @@ -80,13 +85,13 @@ private void processUpsert(RowData row, RequestIndexer indexer) { final String key = createKey.apply(row); if (key != null) { final UpdateRequest updateRequest = - new UpdateRequest(indexGenerator.generate(row), key) + new UpdateRequest(indexGenerator.generate(row), documentType, key) .doc(document, contentType) .upsert(document, contentType); indexer.add(updateRequest); } else { final IndexRequest indexRequest = - new IndexRequest(indexGenerator.generate(row)) + new IndexRequest(indexGenerator.generate(row), documentType) .id(key) .source(document, contentType); indexer.add(indexRequest); @@ -95,7 +100,8 @@ private void processUpsert(RowData row, RequestIndexer indexer) { private void processDelete(RowData row, RequestIndexer indexer) { final String key = createKey.apply(row); - final DeleteRequest deleteRequest = new DeleteRequest(indexGenerator.generate(row), key); + final DeleteRequest deleteRequest = + new DeleteRequest(indexGenerator.generate(row), documentType, key); indexer.add(deleteRequest); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java deleted file mode 100644 index 7fb1e4d4..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; - -import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.common.xcontent.XContentType; - -import javax.annotation.Nullable; - -import java.util.Objects; -import java.util.function.Function; - -/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ -@Internal -class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { - - private static final long serialVersionUID = 1L; - - private final IndexGenerator indexGenerator; - private final String docType; - private final SerializationSchema serializationSchema; - private final XContentType contentType; - private final RequestFactory requestFactory; - private final Function createKey; - - public RowElasticsearchSinkFunction( - IndexGenerator indexGenerator, - @Nullable String docType, // this is deprecated in es 7+ - SerializationSchema serializationSchema, - XContentType contentType, - RequestFactory requestFactory, - Function createKey) { - this.indexGenerator = Preconditions.checkNotNull(indexGenerator); - this.docType = docType; - this.serializationSchema = Preconditions.checkNotNull(serializationSchema); - this.contentType = Preconditions.checkNotNull(contentType); - this.requestFactory = Preconditions.checkNotNull(requestFactory); - this.createKey = Preconditions.checkNotNull(createKey); - } - - @Override - public void open() { - indexGenerator.open(); - } - - @Override - public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) { - switch (element.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - processUpsert(element, indexer); - break; - case UPDATE_BEFORE: - case DELETE: - processDelete(element, indexer); - break; - default: - throw new TableException("Unsupported message kind: " + element.getRowKind()); - } - } - - private void processUpsert(RowData row, RequestIndexer indexer) { - final byte[] document = serializationSchema.serialize(row); - final String key = createKey.apply(row); - if (key != null) { - final UpdateRequest updateRequest = - requestFactory.createUpdateRequest( - indexGenerator.generate(row), docType, key, contentType, document); - indexer.add(updateRequest); - } else { - final IndexRequest indexRequest = - requestFactory.createIndexRequest( - indexGenerator.generate(row), docType, key, contentType, document); - indexer.add(indexRequest); - } - } - - private void processDelete(RowData row, RequestIndexer indexer) { - final String key = createKey.apply(row); - final DeleteRequest deleteRequest = - requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key); - indexer.add(deleteRequest); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; - return Objects.equals(indexGenerator, that.indexGenerator) - && Objects.equals(docType, that.docType) - && Objects.equals(serializationSchema, that.serializationSchema) - && contentType == that.contentType - && Objects.equals(requestFactory, that.requestFactory) - && Objects.equals(createKey, that.createKey); - } - - @Override - public int hashCode() { - return Objects.hash( - indexGenerator, - docType, - serializationSchema, - contentType, - requestFactory, - createKey); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java new file mode 100644 index 00000000..38a167aa --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -0,0 +1,327 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.search.SearchHits; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +import java.io.IOException; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.table.api.Expressions.row; + +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchDynamicSinkBaseITCase { + + abstract String getElasticsearchHttpHostAddress(); + + abstract ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory(); + + abstract Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException; + + abstract SearchHits makeSearchRequest(RestHighLevelClient client, String index) + throws IOException; + + abstract long getTotalSearchHits(SearchHits searchHits); + + abstract TestContext getPrefilledTestContext(String index); + + abstract String getConnectorSql(String index); + + private RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(getElasticsearchHttpHostAddress()))); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + ElasticsearchDynamicSinkFactoryBase sinkFactory = getDynamicSinkFactory(); + + DynamicTableSink.SinkRuntimeProvider runtimeProvider = + sinkFactory + .createDynamicTableSink( + getPrefilledTestContext(index).withSchema(schema).build()) + .getSinkRuntimeProvider(new MockContext()); + + final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; + final Sink sink = sinkProvider.createSink(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).sinkTo(sink); + environment.execute(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, index, "1_2012-12-12T12:12:12"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, index, "1_2012-12-12T12:12:12"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = makeSearchRequest(client, index); + if (getTotalSearchHits(hits) < 2) { + Thread.sleep(200); + } + } while (getTotalSearchHits(hits) < 2 && deadline.hasTimeLeft()); + + if (getTotalSearchHits(hits) < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + Assertions.assertEquals(resultSet, expectedSet); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + Map response = makeGetRequest(client, "dynamic-index-2012-12-12", "1"); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java new file mode 100644 index 00000000..a89e0f96 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -0,0 +1,230 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.function.Executable; + +import java.util.Arrays; +import java.util.Collections; + +/** Tests for validation in {@link ElasticsearchDynamicSinkFactoryBase}. */ +@ExtendWith(TestLoggerExtension.class) +abstract class ElasticsearchDynamicSinkFactoryBaseTest { + + abstract ElasticsearchDynamicSinkFactoryBase createSinkFactory(); + + abstract TestContext createPrefilledTestContext(); + + void assertValidationException(String expectedMessage, Executable executable) { + ValidationException thrown = Assertions.assertThrows(ValidationException.class, executable); + Assertions.assertEquals(expectedMessage, thrown.getMessage()); + } + + @Test + public void validateWrongIndex() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "'index' must not be empty", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION.key(), + "") + .build())); + } + + @Test + public void validateWrongHosts() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "wrong-host") + .build())); + } + + @Test + public void validateWrongFlushSize() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + assertValidationException( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_MAX_SIZE_OPTION + .key(), + "1kb") + .build())); + } + + @Test + public void validateWrongRetries() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build())); + } + + @Test + public void validateWrongMaxActions() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'sink.bulk-flush.max-actions' must be at least 1. Got: -2", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_MAX_ACTIONS_OPTION + .key(), + "-2") + .build())); + } + + @Test + public void validateWrongBackoffDelay() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "Invalid value for option 'sink.bulk-flush.backoff.delay'.", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions + .BULK_FLUSH_BACKOFF_DELAY_OPTION + .key(), + "-1s") + .build())); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical( + "a", + DataTypes.BIGINT() + .notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes + .BIGINT(), + DataTypes + .STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes + .BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes + .FIELD( + "a", + DataTypes + .BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer + .INSTANCE) + .notNull()), + Column.physical( + "g", + DataTypes.BYTES() + .notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList( + "a", "b", "c", "d", "e", + "f", "g")))) + .build())); + } + + @Test + public void validateWrongCredential() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + + assertValidationException( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''", + () -> + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.USERNAME_OPTION.key(), + "username") + .withOption( + ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), + "") + .build())); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java index 8b833216..638f390d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -20,60 +20,18 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.ValidationException; -import org.apache.http.HttpHost; - -import java.util.List; -import java.util.stream.Collectors; - -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; /** Elasticsearch 6 specific configuration. */ @Internal final class Elasticsearch6Configuration extends ElasticsearchConfiguration { - Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { - super(config, classLoader); - } - public List getHosts() { - return config.get(HOSTS_OPTION).stream() - .map(Elasticsearch6Configuration::validateAndParseHostsString) - .collect(Collectors.toList()); + Elasticsearch6Configuration(ReadableConfig config) { + super(config); } - /** - * Parse Hosts String to list. - * - *

Hosts String format was given as following: - * - *

-     *     connector.hosts = http://host_name:9092;http://host_name:9093
-     * 
- */ - private static HttpHost validateAndParseHostsString(String host) { - try { - HttpHost httpHost = HttpHost.create(host); - if (httpHost.getPort() < 0) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", - host, HOSTS_OPTION.key())); - } - - if (httpHost.getSchemeName() == null) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", - host, HOSTS_OPTION.key())); - } - return httpHost; - } catch (Exception e) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", - host, HOSTS_OPTION.key()), - e); - } + public String getDocumentType() { + return config.get(DOCUMENT_TYPE_OPTION); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java new file mode 100644 index 00000000..7620d2ee --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java @@ -0,0 +1,39 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** + * Options specific for the Elasticsearch 6 connector. Public so that the {@link + * org.apache.flink.table.api.TableDescriptor} can access it. + */ +@PublicEvolving +public final class Elasticsearch6ConnectorOptions extends ElasticsearchConnectorOptions { + + private Elasticsearch6ConnectorOptions() {} + + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java deleted file mode 100644 index 508a626e..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ /dev/null @@ -1,317 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; -import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkFunctionProvider; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.StringUtils; - -import org.apache.http.HttpHost; -import org.apache.http.auth.AuthScope; -import org.apache.http.auth.UsernamePasswordCredentials; -import org.apache.http.client.CredentialsProvider; -import org.apache.http.impl.client.BasicCredentialsProvider; -import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.update.UpdateRequest; -import org.elasticsearch.client.RestClientBuilder; -import org.elasticsearch.common.xcontent.XContentType; - -import javax.annotation.Nullable; - -import java.util.List; -import java.util.Objects; - -/** - * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a - * logical description. - */ -@PublicEvolving -final class Elasticsearch6DynamicSink implements DynamicTableSink { - @VisibleForTesting - static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); - - private final EncodingFormat> format; - private final TableSchema schema; - private final Elasticsearch6Configuration config; - - public Elasticsearch6DynamicSink( - EncodingFormat> format, - Elasticsearch6Configuration config, - TableSchema schema) { - this(format, config, schema, (ElasticsearchSink.Builder::new)); - } - - // -------------------------------------------------------------- - // Hack to make configuration testing possible. - // - // The code in this block should never be used outside of tests. - // Having a way to inject a builder we can assert the builder in - // the test. We can not assert everything though, e.g. it is not - // possible to assert flushing on checkpoint, as it is configured - // on the sink itself. - // -------------------------------------------------------------- - - private final ElasticSearchBuilderProvider builderProvider; - - @FunctionalInterface - interface ElasticSearchBuilderProvider { - ElasticsearchSink.Builder createBuilder( - List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); - } - - Elasticsearch6DynamicSink( - EncodingFormat> format, - Elasticsearch6Configuration config, - TableSchema schema, - ElasticSearchBuilderProvider builderProvider) { - this.format = format; - this.schema = schema; - this.config = config; - this.builderProvider = builderProvider; - } - - // -------------------------------------------------------------- - // End of hack to make configuration testing possible - // -------------------------------------------------------------- - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - if (kind != RowKind.UPDATE_BEFORE) { - builder.addContainedKind(kind); - } - } - return builder.build(); - } - - @Override - public SinkFunctionProvider getSinkRuntimeProvider(Context context) { - return () -> { - SerializationSchema format = - this.format.createRuntimeEncoder(context, schema.toRowDataType()); - - final RowElasticsearchSinkFunction upsertFunction = - new RowElasticsearchSinkFunction( - IndexGeneratorFactory.createIndexGenerator(config.getIndex(), schema), - config.getDocumentType(), - format, - XContentType.JSON, - REQUEST_FACTORY, - KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); - - final ElasticsearchSink.Builder builder = - builderProvider.createBuilder(config.getHosts(), upsertFunction); - - builder.setFailureHandler(config.getFailureHandler()); - builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); - builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); - builder.setBulkFlushInterval(config.getBulkFlushInterval()); - builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); - config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); - config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); - config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); - - // we must overwrite the default factory which is defined with a lambda because of a bug - // in shading lambda serialization shading see FLINK-18006 - if (config.getUsername().isPresent() - && config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { - builder.setRestClientFactory( - new AuthRestClientFactory( - config.getPathPrefix().orElse(null), - config.getUsername().get(), - config.getPassword().get())); - } else { - builder.setRestClientFactory( - new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); - } - - final ElasticsearchSink sink = builder.build(); - - if (config.isDisableFlushOnCheckpoint()) { - sink.disableFlushOnCheckpoint(); - } - - return sink; - }; - } - - @Override - public DynamicTableSink copy() { - return this; - } - - @Override - public String asSummaryString() { - return "Elasticsearch6"; - } - - /** Serializable {@link RestClientFactory} used by the sink. */ - @VisibleForTesting - static class DefaultRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - - public DefaultRestClientFactory(@Nullable String pathPrefix) { - this.pathPrefix = pathPrefix; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - DefaultRestClientFactory that = (DefaultRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix); - } - } - - /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ - @VisibleForTesting - static class AuthRestClientFactory implements RestClientFactory { - - private final String pathPrefix; - private final String username; - private final String password; - private transient CredentialsProvider credentialsProvider; - - public AuthRestClientFactory( - @Nullable String pathPrefix, String username, String password) { - this.pathPrefix = pathPrefix; - this.password = password; - this.username = username; - } - - @Override - public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { - if (pathPrefix != null) { - restClientBuilder.setPathPrefix(pathPrefix); - } - if (credentialsProvider == null) { - credentialsProvider = new BasicCredentialsProvider(); - credentialsProvider.setCredentials( - AuthScope.ANY, new UsernamePasswordCredentials(username, password)); - } - restClientBuilder.setHttpClientConfigCallback( - httpAsyncClientBuilder -> - httpAsyncClientBuilder.setDefaultCredentialsProvider( - credentialsProvider)); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - AuthRestClientFactory that = (AuthRestClientFactory) o; - return Objects.equals(pathPrefix, that.pathPrefix) - && Objects.equals(username, that.username) - && Objects.equals(password, that.password); - } - - @Override - public int hashCode() { - return Objects.hash(pathPrefix, username, password); - } - } - - /** - * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the - * sink. - */ - private static class Elasticsearch6RequestFactory implements RequestFactory { - @Override - public UpdateRequest createUpdateRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new UpdateRequest(index, docType, key) - .doc(document, contentType) - .upsert(document, contentType); - } - - @Override - public IndexRequest createIndexRequest( - String index, - String docType, - String key, - XContentType contentType, - byte[] document) { - return new IndexRequest(index, docType, key).source(document, contentType); - } - - @Override - public DeleteRequest createDeleteRequest(String index, String docType, String key) { - return new DeleteRequest(index, docType, key); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; - return Objects.equals(format, that.format) - && Objects.equals(schema, that.schema) - && Objects.equals(config, that.config) - && Objects.equals(builderProvider, that.builderProvider); - } - - @Override - public int hashCode() { - return Objects.hash(format, schema, config, builderProvider); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 32d0ccd4..5f05e608 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -19,152 +19,54 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; +import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder; import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.utils.TableSchemaUtils; import org.apache.flink.util.StringUtils; +import javax.annotation.Nullable; + import java.util.Set; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; -/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ +/** A {@link DynamicTableSinkFactory} for discovering {@link ElasticsearchDynamicSink}. */ @Internal -public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { - private static final Set> requiredOptions = - Stream.of(HOSTS_OPTION, INDEX_OPTION, DOCUMENT_TYPE_OPTION).collect(Collectors.toSet()); - private static final Set> optionalOptions = - Stream.of( - KEY_DELIMITER_OPTION, - FAILURE_HANDLER_OPTION, - FLUSH_ON_CHECKPOINT_OPTION, - BULK_FLASH_MAX_SIZE_OPTION, - BULK_FLUSH_MAX_ACTIONS_OPTION, - BULK_FLUSH_INTERVAL_OPTION, - BULK_FLUSH_BACKOFF_TYPE_OPTION, - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, - BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, - CONNECTION_PATH_PREFIX, - FORMAT_OPTION, - PASSWORD_OPTION, - USERNAME_OPTION) - .collect(Collectors.toSet()); - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - TableSchema tableSchema = context.getCatalogTable().getSchema(); - ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); - final FactoryUtil.TableFactoryHelper helper = - FactoryUtil.createTableFactoryHelper(this, context); +public class Elasticsearch6DynamicSinkFactory extends ElasticsearchDynamicSinkFactoryBase { + private static final String FACTORY_IDENTIFIER = "elasticsearch-6"; - final EncodingFormat> format = - helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); - - helper.validate(); - Configuration configuration = new Configuration(); - context.getCatalogTable().getOptions().forEach(configuration::setString); - Elasticsearch6Configuration config = - new Elasticsearch6Configuration(configuration, context.getClassLoader()); - - validate(config, configuration); - - return new Elasticsearch6DynamicSink( - format, config, TableSchemaUtils.getPhysicalSchema(tableSchema)); + public Elasticsearch6DynamicSinkFactory() { + super(FACTORY_IDENTIFIER, Elasticsearch6SinkBuilder::new); } - private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { - config.getFailureHandler(); // checks if we can instantiate the custom failure handler - config.getHosts(); // validate hosts - validate( - config.getIndex().length() >= 1, - () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); - int maxActions = config.getBulkFlushMaxActions(); - validate( - maxActions == -1 || maxActions >= 1, - () -> - String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); - long maxSize = config.getBulkFlushMaxByteSize(); - long mb1 = 1024 * 1024; - validate( - maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), - () -> - String.format( - "'%s' must be in MB granularity. Got: %s", - BULK_FLASH_MAX_SIZE_OPTION.key(), - originalConfiguration - .get(BULK_FLASH_MAX_SIZE_OPTION) - .toHumanReadableString())); - validate( - config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), - () -> - String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), - config.getBulkFlushBackoffRetries().get())); - if (config.getUsername().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { - validate( - config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), - () -> - String.format( - "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", - USERNAME_OPTION.key(), - PASSWORD_OPTION.key(), - config.getUsername().get(), - config.getPassword().orElse(""))); - } + @Override + ElasticsearchConfiguration getConfiguration(Context context) { + return new Elasticsearch6Configuration( + Configuration.fromMap(context.getCatalogTable().getOptions())); } - private static void validate(boolean condition, Supplier message) { - if (!condition) { - throw new ValidationException(message.get()); - } + @Nullable + @Override + String getDocumentType(Context context) { + Elasticsearch6Configuration config = + (Elasticsearch6Configuration) getConfiguration(context); + return config.getDocumentType(); } @Override - public String factoryIdentifier() { - return "elasticsearch-6"; + void validateConfiguration(ElasticsearchConfiguration config) { + super.validateConfiguration(config); + Elasticsearch6Configuration configuration = (Elasticsearch6Configuration) config; + validate( + !StringUtils.isNullOrWhitespaceOnly(configuration.getDocumentType()), + () -> String.format("'%s' must not be empty", DOCUMENT_TYPE_OPTION.key())); } @Override public Set> requiredOptions() { + Set> requiredOptions = super.requiredOptions(); + requiredOptions.add(DOCUMENT_TYPE_OPTION); return requiredOptions; } - - @Override - public Set> optionalOptions() { - return optionalOptions; - } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index e99abbe9..3a384edf 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -18,233 +18,38 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.typeutils.base.VoidSerializer; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.util.TestLogger; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.util.Arrays; -import java.util.Collections; +import org.junit.jupiter.api.Test; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ -public class Elasticsearch6DynamicSinkFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +public class Elasticsearch6DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { + @Override + ElasticsearchDynamicSinkFactoryBase createSinkFactory() { + return new Elasticsearch6DynamicSinkFactory(); + } + + @Override + TestContext createPrefilledTestContext() { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption(Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345"); + } @Test public void validateEmptyConfiguration() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); - thrown.expect(ValidationException.class); - thrown.expectMessage( + assertValidationException( "One or more required options are missing.\n" + "\n" + "Missing required options are:\n" + "\n" + "document-type\n" + "hosts\n" - + "index"); - sinkFactory.createDynamicTableSink(context().build()); - } - - @Test - public void validateWrongIndex() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'index' must not be empty"); - sinkFactory.createDynamicTableSink( - context() - .withOption("index", "") - .withOption("document-type", "MyType") - .withOption("hosts", "http://localhost:12345") - .build()); - } - - @Test - public void validateWrongHosts() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); - sinkFactory.createDynamicTableSink( - context() - .withOption("index", "MyIndex") - .withOption("document-type", "MyType") - .withOption("hosts", "wrong-host") - .build()); - } - - @Test - public void validateWrongFlushSize() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), - "1kb") - .build()); - } - - @Test - public void validateWrongRetries() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION - .key(), - "0") - .build()); - } - - @Test - public void validateWrongMaxActions() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - "-2") - .build()); - } - - @Test - public void validateWrongBackoffDelay() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), - "-1s") - .build()); - } - - @Test - public void validatePrimaryKeyOnIllegalColumn() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "The table has a primary key on columns of illegal types: " - + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" - + " Elasticsearch sink does not support primary keys on columns of types: " - + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); - sinkFactory.createDynamicTableSink( - context() - .withSchema( - new ResolvedSchema( - Arrays.asList( - Column.physical("a", DataTypes.BIGINT().notNull()), - Column.physical( - "b", - DataTypes.ARRAY( - DataTypes.BIGINT() - .notNull()) - .notNull()), - Column.physical( - "c", - DataTypes.MAP( - DataTypes.BIGINT(), - DataTypes.STRING()) - .notNull()), - Column.physical( - "d", - DataTypes.MULTISET( - DataTypes.BIGINT() - .notNull()) - .notNull()), - Column.physical( - "e", - DataTypes.ROW( - DataTypes.FIELD( - "a", - DataTypes.BIGINT())) - .notNull()), - Column.physical( - "f", - DataTypes.RAW( - Void.class, - VoidSerializer.INSTANCE) - .notNull()), - Column.physical("g", DataTypes.BYTES().notNull())), - Collections.emptyList(), - UniqueConstraint.primaryKey( - "name", - Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), - "1s") - .build()); - } - - @Test - public void validateWrongCredential() { - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") - .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") - .build()); + + "index", + () -> sinkFactory.createDynamicTableSink(context().build())); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 0ebc52b5..4d2bb5a2 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -18,382 +18,79 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkFunctionProvider; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; -import org.apache.flink.util.TestLogger; import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.transport.client.PreBuiltTransportClient; -import org.junit.ClassRule; -import org.junit.Test; import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; -import java.time.Duration; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; +import java.io.IOException; import java.util.Map; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; -import static org.apache.flink.table.api.Expressions.row; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertThat; -/** IT tests for {@link Elasticsearch6DynamicSink}. */ -public class Elasticsearch6DynamicSinkITCase extends TestLogger { +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@Testcontainers +public class Elasticsearch6DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); + private static final String DOCUMENT_TYPE = "MyType"; - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); - } - - @Test - public void testWritingDocuments() throws Exception { - ResolvedSchema schema = - new ResolvedSchema( - Arrays.asList( - Column.physical("a", DataTypes.BIGINT().notNull()), - Column.physical("b", DataTypes.TIME()), - Column.physical("c", DataTypes.STRING().notNull()), - Column.physical("d", DataTypes.FLOAT()), - Column.physical("e", DataTypes.TINYINT().notNull()), - Column.physical("f", DataTypes.DATE()), - Column.physical("g", DataTypes.TIMESTAMP().notNull())), - Collections.emptyList(), - UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); - GenericRowData rowData = - GenericRowData.of( - 1L, - 12345, - StringData.fromString("ABCDE"), - 12.12f, - (byte) 2, - 12345, - TimestampData.fromLocalDateTime( - LocalDateTime.parse("2012-12-12T12:12:12"))); - - String index = "writing-documents"; - String myType = "MyType"; - Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); - - SinkFunctionProvider sinkRuntimeProvider = - (SinkFunctionProvider) - sinkFactory - .createDynamicTableSink( - context() - .withSchema(schema) - .withOption( - ElasticsearchConnectorOptions.INDEX_OPTION - .key(), - index) - .withOption( - ElasticsearchConnectorOptions - .DOCUMENT_TYPE_OPTION - .key(), - myType) - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION - .key(), - elasticsearchContainer.getHttpHostAddress()) - .withOption( - ElasticsearchConnectorOptions - .FLUSH_ON_CHECKPOINT_OPTION - .key(), - "false") - .build()) - .getSinkRuntimeProvider(new MockContext()); - - SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); - StreamExecutionEnvironment environment = - StreamExecutionEnvironment.getExecutionEnvironment(); - environment.setParallelism(4); - - rowData.setRowKind(RowKind.UPDATE_AFTER); - environment.fromElements(rowData).addSink(sinkFunction); - environment.execute(); + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_6); - Client client = getClient(); - Map response = - client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) - .actionGet() - .getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); } - @Test - public void testWritingDocumentsFromTableApi() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "table-api"; - String myType = "MyType"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIME,\n" - + "c STRING NOT NULL,\n" - + "d FLOAT,\n" - + "e TINYINT NOT NULL,\n" - + "f DATE,\n" - + "g TIMESTAMP NOT NULL,\n" - + "h as a + 2,\n" - + "PRIMARY KEY (a, g) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + String.format( - "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") - + ")"); - - tableEnvironment - .fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - Map response = - client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) - .actionGet() - .getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory() { + return new Elasticsearch6DynamicSinkFactory(); } - @Test - public void testWritingDocumentsNoPrimaryKey() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "no-primary-key"; - String myType = "MyType"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIME,\n" - + "c STRING NOT NULL,\n" - + "d FLOAT,\n" - + "e TINYINT NOT NULL,\n" - + "f DATE,\n" - + "g TIMESTAMP NOT NULL\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + String.format( - "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") - + ")"); - - tableEnvironment - .fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 2L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "FGHIJK", - 13.13f, - (byte) 4, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2013-12-12T13:13:13"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - - // search API does not return documents that were not indexed, we might need to query - // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); - SearchHits hits; - do { - hits = client.prepareSearch(index).execute().actionGet().getHits(); - if (hits.getTotalHits() < 2) { - Thread.sleep(200); - } - } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); - - if (hits.getTotalHits() < 2) { - throw new AssertionError("Could not retrieve results from Elasticsearch."); - } - - HashSet> resultSet = new HashSet<>(); - resultSet.add(hits.getAt(0).getSourceAsMap()); - resultSet.add(hits.getAt(1).getSourceAsMap()); - Map expectedMap1 = new HashMap<>(); - expectedMap1.put("a", 1); - expectedMap1.put("b", "00:00:12"); - expectedMap1.put("c", "ABCDE"); - expectedMap1.put("d", 12.12d); - expectedMap1.put("e", 2); - expectedMap1.put("f", "2003-10-20"); - expectedMap1.put("g", "2012-12-12 12:12:12"); - Map expectedMap2 = new HashMap<>(); - expectedMap2.put("a", 2); - expectedMap2.put("b", "00:00:12"); - expectedMap2.put("c", "FGHIJK"); - expectedMap2.put("d", 13.13d); - expectedMap2.put("e", 4); - expectedMap2.put("f", "2003-10-20"); - expectedMap2.put("g", "2013-12-12 13:13:13"); - HashSet> expectedSet = new HashSet<>(); - expectedSet.add(expectedMap1); - expectedSet.add(expectedMap2); - assertThat(resultSet, equalTo(expectedSet)); + @Override + Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException { + return client.get(new GetRequest(index, DOCUMENT_TYPE, id)).getSource(); } - @Test - public void testWritingDocumentsWithDynamicIndex() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "dynamic-index-{b|yyyy-MM-dd}"; - String myType = "MyType"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIMESTAMP NOT NULL,\n" - + "PRIMARY KEY (a) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) - + String.format( - "'%s'='%s',\n", - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + String.format( - "'%s'='%s'\n", - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), - "false") - + ")"); - - tableEnvironment - .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - Map response = - client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")) - .actionGet() - .getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + SearchHits makeSearchRequest(RestHighLevelClient client, String index) throws IOException { + return client.search(new SearchRequest(index)).getHits(); } - private static class MockContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } + @Override + long getTotalSearchHits(SearchHits hits) { + return hits.getTotalHits(); + } - @Override - public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { - return null; - } + @Override + TestContext getPrefilledTestContext(String index) { + return context() + .withOption(Elasticsearch6ConnectorOptions.INDEX_OPTION.key(), index) + .withOption( + Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOCUMENT_TYPE) + .withOption( + Elasticsearch6ConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType consumedDataType) { - return null; - } + @Override + String getConnectorSql(String index) { + return String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", Elasticsearch6ConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOCUMENT_TYPE) + + String.format( + "'%s'='%s'\n", + Elasticsearch6ConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java deleted file mode 100644 index d1af3ec7..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java +++ /dev/null @@ -1,294 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; -import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; -import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.util.TestLogger; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.ActionRequest; -import org.junit.Test; -import org.mockito.Mockito; - -import java.util.List; - -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.verify; - -/** Tests for {@link Elasticsearch6DynamicSink} parameters. */ -public class Elasticsearch6DynamicSinkTest extends TestLogger { - - private static final String FIELD_KEY = "key"; - private static final String FIELD_FRUIT_NAME = "fruit_name"; - private static final String FIELD_COUNT = "count"; - private static final String FIELD_TS = "ts"; - - private static final String HOSTNAME = "host1"; - private static final int PORT = 1234; - private static final String SCHEMA = "https"; - private static final String INDEX = "MyIndex"; - private static final String DOC_TYPE = "MyType"; - private static final String USERNAME = "username"; - private static final String PASSWORD = "password"; - - @Test - public void testBuilder() { - final TableSchema schema = createTestSchema(); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = - new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration( - getConfig(), this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(true); - verify(provider.builderSpy) - .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); - verify(provider.builderSpy).setBulkFlushBackoffDelay(123); - verify(provider.builderSpy).setBulkFlushBackoffRetries(3); - verify(provider.builderSpy).setBulkFlushInterval(100); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); - verify(provider.builderSpy) - .setRestClientFactory( - new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); - verify(provider.sinkSpy).disableFlushOnCheckpoint(); - } - - @Test - public void testDefaultConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = - new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration( - configuration, this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy) - .setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - @Test - public void testAuthConfig() { - final TableSchema schema = createTestSchema(); - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); - configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); - - BuilderProvider provider = new BuilderProvider(); - final Elasticsearch6DynamicSink testSink = - new Elasticsearch6DynamicSink( - new DummyEncodingFormat(), - new Elasticsearch6Configuration( - configuration, this.getClass().getClassLoader()), - schema, - provider); - - testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); - - verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); - verify(provider.builderSpy).setBulkFlushBackoff(false); - verify(provider.builderSpy).setBulkFlushInterval(1000); - verify(provider.builderSpy).setBulkFlushMaxActions(1000); - verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); - verify(provider.builderSpy) - .setRestClientFactory( - new Elasticsearch6DynamicSink.AuthRestClientFactory( - null, USERNAME, PASSWORD)); - verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); - } - - private Configuration getConfig() { - Configuration configuration = new Configuration(); - configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); - configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); - configuration.setString( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - SCHEMA + "://" + HOSTNAME + ":" + PORT); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); - configuration.setString( - ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); - configuration.setString( - ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); - configuration.setString( - ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), - DummyFailureHandler.class.getName()); - configuration.setString( - ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); - return configuration; - } - - private static class BuilderProvider - implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { - public ElasticsearchSink.Builder builderSpy; - public ElasticsearchSink sinkSpy; - - @Override - public ElasticsearchSink.Builder createBuilder( - List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { - builderSpy = - Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); - doAnswer( - invocation -> { - sinkSpy = - Mockito.spy( - (ElasticsearchSink) - invocation.callRealMethod()); - return sinkSpy; - }) - .when(builderSpy) - .build(); - - return builderSpy; - } - } - - private TableSchema createTestSchema() { - return TableSchema.builder() - .field(FIELD_KEY, DataTypes.BIGINT()) - .field(FIELD_FRUIT_NAME, DataTypes.STRING()) - .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) - .field(FIELD_TS, DataTypes.TIMESTAMP(3)) - .build(); - } - - private static class DummySerializationSchema implements SerializationSchema { - - private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); - - @Override - public byte[] serialize(RowData element) { - return new byte[0]; - } - } - - private static class DummyEncodingFormat - implements EncodingFormat> { - @Override - public SerializationSchema createRuntimeEncoder( - DynamicTableSink.Context context, DataType consumedDataType) { - return DummySerializationSchema.INSTANCE; - } - - @Override - public ChangelogMode getChangelogMode() { - return null; - } - } - - private static class MockSinkContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { - return null; - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType consumedDataType) { - return null; - } - } - - /** Custom failure handler for testing. */ - public static class DummyFailureHandler implements ActionRequestFailureHandler { - - @Override - public void onFailure( - ActionRequest action, - Throwable failure, - int restStatusCode, - RequestIndexer indexer) { - // do nothing - } - - @Override - public boolean equals(Object o) { - return o instanceof DummyFailureHandler; - } - - @Override - public int hashCode() { - return DummyFailureHandler.class.hashCode(); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java deleted file mode 100644 index 276f77c0..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; -import org.apache.flink.table.api.ValidationException; - -import org.apache.http.HttpHost; - -import java.time.Duration; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; - -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.DELIVERY_GUARANTEE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.USERNAME_OPTION; -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** Elasticsearch 7 specific configuration. */ -@Internal -final class Elasticsearch7Configuration { - protected final ReadableConfig config; - - Elasticsearch7Configuration(ReadableConfig config) { - this.config = checkNotNull(config); - } - - public int getBulkFlushMaxActions() { - return config.get(BULK_FLUSH_MAX_ACTIONS_OPTION); - } - - public long getBulkFlushMaxByteSize() { - return config.get(BULK_FLUSH_MAX_SIZE_OPTION).getBytes(); - } - - public long getBulkFlushInterval() { - return config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); - } - - public DeliveryGuarantee getDeliveryGuarantee() { - return config.get(DELIVERY_GUARANTEE_OPTION); - } - - public Optional getUsername() { - return config.getOptional(USERNAME_OPTION); - } - - public Optional getPassword() { - return config.getOptional(PASSWORD_OPTION); - } - - public Optional getBulkFlushBackoffType() { - return config.getOptional(BULK_FLUSH_BACKOFF_TYPE_OPTION); - } - - public Optional getBulkFlushBackoffRetries() { - return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); - } - - public Optional getBulkFlushBackoffDelay() { - return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); - } - - public String getIndex() { - return config.get(INDEX_OPTION); - } - - public String getKeyDelimiter() { - return config.get(KEY_DELIMITER_OPTION); - } - - public Optional getPathPrefix() { - return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); - } - - public List getHosts() { - return config.get(HOSTS_OPTION).stream() - .map(Elasticsearch7Configuration::validateAndParseHostsString) - .collect(Collectors.toList()); - } - - private static HttpHost validateAndParseHostsString(String host) { - try { - HttpHost httpHost = HttpHost.create(host); - if (httpHost.getPort() < 0) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", - host, HOSTS_OPTION.key())); - } - - if (httpHost.getSchemeName() == null) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", - host, HOSTS_OPTION.key())); - } - return httpHost; - } catch (Exception e) { - throw new ValidationException( - String.format( - "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", - host, HOSTS_OPTION.key()), - e); - } - } -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java deleted file mode 100644 index 6048e124..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7ConnectorOptions.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.connectors.elasticsearch.table; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; - -import java.time.Duration; -import java.util.List; - -/** Options for the Elasticsearch connector. */ -@PublicEvolving -public class Elasticsearch7ConnectorOptions { - - private Elasticsearch7ConnectorOptions() {} - - public static final ConfigOption> HOSTS_OPTION = - ConfigOptions.key("hosts") - .stringType() - .asList() - .noDefaultValue() - .withDescription("Elasticsearch hosts to connect to."); - - public static final ConfigOption INDEX_OPTION = - ConfigOptions.key("index") - .stringType() - .noDefaultValue() - .withDescription("Elasticsearch index for every record."); - - public static final ConfigOption PASSWORD_OPTION = - ConfigOptions.key("password") - .stringType() - .noDefaultValue() - .withDescription("Password used to connect to Elasticsearch instance."); - - public static final ConfigOption USERNAME_OPTION = - ConfigOptions.key("username") - .stringType() - .noDefaultValue() - .withDescription("Username used to connect to Elasticsearch instance."); - - public static final ConfigOption KEY_DELIMITER_OPTION = - ConfigOptions.key("document-id.key-delimiter") - .stringType() - .defaultValue("_") - .withDescription( - "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); - - public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = - ConfigOptions.key("sink.bulk-flush.max-actions") - .intType() - .defaultValue(1000) - .withDescription("Maximum number of actions to buffer for each bulk request."); - - public static final ConfigOption BULK_FLUSH_MAX_SIZE_OPTION = - ConfigOptions.key("sink.bulk-flush.max-size") - .memoryType() - .defaultValue(MemorySize.parse("2mb")) - .withDescription("Maximum size of buffered actions per bulk request"); - - public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = - ConfigOptions.key("sink.bulk-flush.interval") - .durationType() - .defaultValue(Duration.ofSeconds(1)) - .withDescription("Bulk flush interval"); - - public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.strategy") - .enumType(FlushBackoffType.class) - .noDefaultValue() - .withDescription("Backoff strategy"); - - public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.max-retries") - .intType() - .noDefaultValue() - .withDescription("Maximum number of retries."); - - public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = - ConfigOptions.key("sink.bulk-flush.backoff.delay") - .durationType() - .noDefaultValue() - .withDescription("Delay between each backoff attempt."); - - public static final ConfigOption CONNECTION_PATH_PREFIX_OPTION = - ConfigOptions.key("connection.path-prefix") - .stringType() - .noDefaultValue() - .withDescription("Prefix string to be added to every REST communication."); - - public static final ConfigOption FORMAT_OPTION = - ConfigOptions.key("format") - .stringType() - .defaultValue("json") - .withDescription( - "The format must produce a valid JSON document. " - + "Please refer to the documentation on formats for more details."); - - public static final ConfigOption DELIVERY_GUARANTEE_OPTION = - ConfigOptions.key("sink.delivery-guarantee") - .enumType(DeliveryGuarantee.class) - .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) - .withDescription("Optional delivery guarantee when committing."); -} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 0646d26f..0d4b1069 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -19,174 +19,15 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.connector.format.EncodingFormat; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.data.RowData; +import org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder; import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.FactoryUtil; -import org.apache.flink.table.factories.SerializationFormatFactory; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.util.StringUtils; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.DELIVERY_GUARANTEE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7ConnectorOptions.USERNAME_OPTION; - -/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ +/** A {@link DynamicTableSinkFactory} for discovering {@link ElasticsearchDynamicSink}. */ @Internal -public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { - private static final Set> requiredOptions = - Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); - private static final Set> optionalOptions = - Stream.of( - KEY_DELIMITER_OPTION, - BULK_FLUSH_MAX_SIZE_OPTION, - BULK_FLUSH_MAX_ACTIONS_OPTION, - BULK_FLUSH_INTERVAL_OPTION, - BULK_FLUSH_BACKOFF_TYPE_OPTION, - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, - BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_PATH_PREFIX_OPTION, - FORMAT_OPTION, - DELIVERY_GUARANTEE_OPTION, - PASSWORD_OPTION, - USERNAME_OPTION) - .collect(Collectors.toSet()); - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - DataType physicalRowDataType = context.getPhysicalRowDataType(); - int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); - if (primaryKeyIndexes.length != 0) { - DataType pkDataType = DataType.projectFields(physicalRowDataType, primaryKeyIndexes); - - ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); - } - - ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); - List primaryKeyLogicalTypesWithIndex = - Arrays.stream(primaryKeyIndexes) - .mapToObj( - index -> { - Optional column = resolvedSchema.getColumn(index); - if (!column.isPresent()) { - throw new IllegalStateException( - String.format( - "No primary key column found with index '%s'.", - index)); - } - LogicalType logicalType = - column.get().getDataType().getLogicalType(); - return new LogicalTypeWithIndex(index, logicalType); - }) - .collect(Collectors.toList()); - - final FactoryUtil.TableFactoryHelper helper = - FactoryUtil.createTableFactoryHelper(this, context); - - final EncodingFormat> format = - helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); - - helper.validate(); - Configuration configuration = new Configuration(); - context.getCatalogTable().getOptions().forEach(configuration::setString); - Elasticsearch7Configuration config = new Elasticsearch7Configuration(configuration); - - validate(config, configuration); - - return new Elasticsearch7DynamicSink( - format, config, primaryKeyLogicalTypesWithIndex, physicalRowDataType); - } - - private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { - config.getHosts(); // validate hosts - validate( - config.getIndex().length() >= 1, - () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); - int maxActions = config.getBulkFlushMaxActions(); - validate( - maxActions == -1 || maxActions >= 1, - () -> - String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); - long maxSize = config.getBulkFlushMaxByteSize(); - long mb1 = 1024 * 1024; - validate( - maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), - () -> - String.format( - "'%s' must be in MB granularity. Got: %s", - BULK_FLUSH_MAX_SIZE_OPTION.key(), - originalConfiguration - .get(BULK_FLUSH_MAX_SIZE_OPTION) - .toHumanReadableString())); - validate( - config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), - () -> - String.format( - "'%s' must be at least 1. Got: %s", - BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), - config.getBulkFlushBackoffRetries().get())); - if (config.getUsername().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { - validate( - config.getPassword().isPresent() - && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), - () -> - String.format( - "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", - USERNAME_OPTION.key(), - PASSWORD_OPTION.key(), - config.getUsername().get(), - config.getPassword().orElse(""))); - } - } - - private static void validate(boolean condition, Supplier message) { - if (!condition) { - throw new ValidationException(message.get()); - } - } - - @Override - public String factoryIdentifier() { - return "elasticsearch-7"; - } - - @Override - public Set> requiredOptions() { - return requiredOptions; - } +public class Elasticsearch7DynamicSinkFactory extends ElasticsearchDynamicSinkFactoryBase { + private static final String FACTORY_IDENTIFIER = "elasticsearch-7"; - @Override - public Set> optionalOptions() { - return optionalOptions; + public Elasticsearch7DynamicSinkFactory() { + super(FACTORY_IDENTIFIER, Elasticsearch7SinkBuilder::new); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index 4f40c1cc..ee217d26 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -18,215 +18,36 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.typeutils.base.VoidSerializer; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.util.TestLogger; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; - -import java.util.Arrays; -import java.util.Collections; +import org.junit.jupiter.api.Test; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ -public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { - @Rule public ExpectedException thrown = ExpectedException.none(); +public class Elasticsearch7DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { + @Override + ElasticsearchDynamicSinkFactoryBase createSinkFactory() { + return new Elasticsearch7DynamicSinkFactory(); + } + + @Override + TestContext createPrefilledTestContext() { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), "http://localhost:12345"); + } @Test public void validateEmptyConfiguration() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); - thrown.expect(ValidationException.class); - thrown.expectMessage( + assertValidationException( "One or more required options are missing.\n" + "\n" + "Missing required options are:\n" + "\n" + "hosts\n" - + "index"); - sinkFactory.createDynamicTableSink(context().build()); - } - - @Test - public void validateWrongIndex() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'index' must not be empty"); - sinkFactory.createDynamicTableSink( - context() - .withOption("index", "") - .withOption("hosts", "http://localhost:12345") - .build()); - } - - @Test - public void validateWrongHosts() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); - sinkFactory.createDynamicTableSink( - context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); - } - - @Test - public void validateWrongFlushSize() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), - "1kb") - .build()); - } - - @Test - public void validateWrongRetries() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION - .key(), - "0") - .build()); - } - - @Test - public void validateWrongMaxActions() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), - "-2") - .build()); - } - - @Test - public void validateWrongBackoffDelay() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), - "-1s") - .build()); - } - - @Test - public void validatePrimaryKeyOnIllegalColumn() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "The table has a primary key on columns of illegal types: " - + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY]."); - sinkFactory.createDynamicTableSink( - context() - .withSchema( - new ResolvedSchema( - Arrays.asList( - Column.physical("a", DataTypes.BIGINT().notNull()), - Column.physical( - "b", - DataTypes.ARRAY( - DataTypes.BIGINT() - .notNull()) - .notNull()), - Column.physical( - "c", - DataTypes.MAP( - DataTypes.BIGINT(), - DataTypes.STRING()) - .notNull()), - Column.physical( - "d", - DataTypes.MULTISET( - DataTypes.BIGINT() - .notNull()) - .notNull()), - Column.physical( - "e", - DataTypes.ROW( - DataTypes.FIELD( - "a", - DataTypes.BIGINT())) - .notNull()), - Column.physical( - "f", - DataTypes.RAW( - Void.class, - VoidSerializer.INSTANCE) - .notNull()), - Column.physical("g", DataTypes.BYTES().notNull())), - Collections.emptyList(), - UniqueConstraint.primaryKey( - "name", - Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption( - ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), - "1s") - .build()); - } - - @Test - public void validateWrongCredential() { - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - thrown.expect(ValidationException.class); - thrown.expectMessage( - "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); - sinkFactory.createDynamicTableSink( - context() - .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") - .withOption( - ElasticsearchConnectorOptions.HOSTS_OPTION.key(), - "http://localhost:1234") - .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") - .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") - .build()); + + "index", + () -> sinkFactory.createDynamicTableSink(context().build())); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 03d20366..8def2d26 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -18,340 +18,73 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; -import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.sink.Sink; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkProvider; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.types.RowKind; import org.apache.flink.util.DockerImageVersions; -import org.apache.flink.util.TestLogger; import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.transport.client.PreBuiltTransportClient; -import org.junit.ClassRule; -import org.junit.Test; import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; -import java.time.Duration; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; +import java.io.IOException; import java.util.Map; import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; -import static org.apache.flink.table.api.Expressions.row; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertThat; -/** IT tests for {@link Elasticsearch7DynamicSink}. */ -public class Elasticsearch7DynamicSinkITCase extends TestLogger { +/** IT tests for {@link ElasticsearchDynamicSink}. */ +@Testcontainers +public class Elasticsearch7DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { - @ClassRule - public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + @Container + private static final ElasticsearchContainer ES_CONTAINER = + new ElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_7); - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + @Override + String getElasticsearchHttpHostAddress() { + return ES_CONTAINER.getHttpHostAddress(); } - @Test - public void testWritingDocuments() throws Exception { - ResolvedSchema schema = - new ResolvedSchema( - Arrays.asList( - Column.physical("a", DataTypes.BIGINT().notNull()), - Column.physical("b", DataTypes.TIME()), - Column.physical("c", DataTypes.STRING().notNull()), - Column.physical("d", DataTypes.FLOAT()), - Column.physical("e", DataTypes.TINYINT().notNull()), - Column.physical("f", DataTypes.DATE()), - Column.physical("g", DataTypes.TIMESTAMP().notNull())), - Collections.emptyList(), - UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); - - GenericRowData rowData = - GenericRowData.of( - 1L, - 12345, - StringData.fromString("ABCDE"), - 12.12f, - (byte) 2, - 12345, - TimestampData.fromLocalDateTime( - LocalDateTime.parse("2012-12-12T12:12:12"))); - - String index = "writing-documents"; - Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); - - DynamicTableSink.SinkRuntimeProvider runtimeProvider = - sinkFactory - .createDynamicTableSink( - context() - .withSchema(schema) - .withOption( - Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), - index) - .withOption( - Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - .build()) - .getSinkRuntimeProvider(new MockContext()); - - final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; - final Sink sink = sinkProvider.createSink(); - StreamExecutionEnvironment environment = - StreamExecutionEnvironment.getExecutionEnvironment(); - environment.setParallelism(4); - - rowData.setRowKind(RowKind.UPDATE_AFTER); - environment.fromElements(rowData).sinkTo(sink); - environment.execute(); - - Client client = getClient(); - Map response = - client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + ElasticsearchDynamicSinkFactoryBase getDynamicSinkFactory() { + return new Elasticsearch7DynamicSinkFactory(); } - @Test - public void testWritingDocumentsFromTableApi() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "table-api"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIME,\n" - + "c STRING NOT NULL,\n" - + "d FLOAT,\n" - + "e TINYINT NOT NULL,\n" - + "f DATE,\n" - + "g TIMESTAMP NOT NULL," - + "h as a + 2,\n" - + "PRIMARY KEY (a, g) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") - + String.format( - "'%s'='%s',\n", - Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s'\n", - Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + ")"); - - tableEnvironment - .fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - Map response = - client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "00:00:12"); - expectedMap.put("c", "ABCDE"); - expectedMap.put("d", 12.12d); - expectedMap.put("e", 2); - expectedMap.put("f", "2003-10-20"); - expectedMap.put("g", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + Map makeGetRequest(RestHighLevelClient client, String index, String id) + throws IOException { + return client.get(new GetRequest(index, id), RequestOptions.DEFAULT).getSource(); } - @Test - public void testWritingDocumentsNoPrimaryKey() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "no-primary-key"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIME,\n" - + "c STRING NOT NULL,\n" - + "d FLOAT,\n" - + "e TINYINT NOT NULL,\n" - + "f DATE,\n" - + "g TIMESTAMP NOT NULL\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") - + String.format( - "'%s'='%s',\n", - Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s'\n", - Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + ")"); - - tableEnvironment - .fromValues( - row( - 1L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "ABCDE", - 12.12f, - (byte) 2, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2012-12-12T12:12:12")), - row( - 2L, - LocalTime.ofNanoOfDay(12345L * 1_000_000L), - "FGHIJK", - 13.13f, - (byte) 4, - LocalDate.ofEpochDay(12345), - LocalDateTime.parse("2013-12-12T13:13:13"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - - // search API does not return documents that were not indexed, we might need to query - // the index a few times - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); - SearchHits hits; - do { - hits = client.prepareSearch(index).execute().actionGet().getHits(); - if (hits.getTotalHits().value < 2) { - Thread.sleep(200); - } - } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); - - if (hits.getTotalHits().value < 2) { - throw new AssertionError("Could not retrieve results from Elasticsearch."); - } - - HashSet> resultSet = new HashSet<>(); - resultSet.add(hits.getAt(0).getSourceAsMap()); - resultSet.add(hits.getAt(1).getSourceAsMap()); - Map expectedMap1 = new HashMap<>(); - expectedMap1.put("a", 1); - expectedMap1.put("b", "00:00:12"); - expectedMap1.put("c", "ABCDE"); - expectedMap1.put("d", 12.12d); - expectedMap1.put("e", 2); - expectedMap1.put("f", "2003-10-20"); - expectedMap1.put("g", "2012-12-12 12:12:12"); - Map expectedMap2 = new HashMap<>(); - expectedMap2.put("a", 2); - expectedMap2.put("b", "00:00:12"); - expectedMap2.put("c", "FGHIJK"); - expectedMap2.put("d", 13.13d); - expectedMap2.put("e", 4); - expectedMap2.put("f", "2003-10-20"); - expectedMap2.put("g", "2013-12-12 13:13:13"); - HashSet> expectedSet = new HashSet<>(); - expectedSet.add(expectedMap1); - expectedSet.add(expectedMap2); - assertThat(resultSet, equalTo(expectedSet)); + @Override + SearchHits makeSearchRequest(RestHighLevelClient client, String index) throws IOException { + return client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); } - @Test - public void testWritingDocumentsWithDynamicIndex() throws Exception { - TableEnvironment tableEnvironment = - TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - - String index = "dynamic-index-{b|yyyy-MM-dd}"; - tableEnvironment.executeSql( - "CREATE TABLE esTable (" - + "a BIGINT NOT NULL,\n" - + "b TIMESTAMP NOT NULL,\n" - + "PRIMARY KEY (a) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") - + String.format( - "'%s'='%s',\n", - Elasticsearch7ConnectorOptions.INDEX_OPTION.key(), index) - + String.format( - "'%s'='%s'\n", - Elasticsearch7ConnectorOptions.HOSTS_OPTION.key(), - elasticsearchContainer.getHttpHostAddress()) - + ")"); - - tableEnvironment - .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) - .executeInsert("esTable") - .await(); - - Client client = getClient(); - Map response = - client.get(new GetRequest("dynamic-index-2012-12-12", "1")).actionGet().getSource(); - Map expectedMap = new HashMap<>(); - expectedMap.put("a", 1); - expectedMap.put("b", "2012-12-12 12:12:12"); - assertThat(response, equalTo(expectedMap)); + @Override + long getTotalSearchHits(SearchHits searchHits) { + return searchHits.getTotalHits().value; } - private static class MockContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { - return null; - } + @Override + TestContext getPrefilledTestContext(String index) { + return context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); + } - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType consumedDataType) { - return null; - } + @Override + String getConnectorSql(String index) { + return String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + ES_CONTAINER.getHttpHostAddress()); } } From 2968a585159ba16813fdf79365e9ba9de6b4fc4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 21 Oct 2021 13:54:57 +0200 Subject: [PATCH 164/207] [FLINK-24325][connectors/elasticsearch] Migrate KeyExtractorTest to JUnit 5 and remove dependency on deprecated TableSchema --- .../elasticsearch/table/KeyExtractor.java | 33 ----- .../elasticsearch/table/KeyExtractorTest.java | 114 ++++++++++-------- 2 files changed, 63 insertions(+), 84 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java index fb833279..888994d9 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; @@ -30,10 +29,7 @@ import java.time.LocalDate; import java.time.LocalTime; import java.time.Period; -import java.util.HashMap; import java.util.List; -import java.util.Map; -import java.util.function.Function; /** An extractor for a Elasticsearch key from a {@link RowData}. */ @Internal @@ -81,35 +77,6 @@ public int getIndex() { } } - @Deprecated - public static Function createKeyExtractor( - TableSchema schema, String keyDelimiter) { - return schema.getPrimaryKey() - .map( - key -> { - Map namesToColumns = new HashMap<>(); - List tableColumns = schema.getTableColumns(); - for (int i = 0; i < schema.getFieldCount(); i++) { - TableColumn column = tableColumns.get(i); - namesToColumns.put( - column.getName(), new ColumnWithIndex(column, i)); - } - - FieldFormatter[] fieldFormatters = - key.getColumns().stream() - .map(namesToColumns::get) - .map( - column -> - toFormatter( - column.index, column.getType())) - .toArray(FieldFormatter[]::new); - - return (Function) - new KeyExtractor(fieldFormatters, keyDelimiter); - }) - .orElseGet(() -> (Function & Serializable) (row) -> null); - } - public static SerializableFunction createKeyExtractor( List primaryKeyTypesWithIndex, String keyDelimiter) { if (!primaryKeyTypesWithIndex.isEmpty()) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java index bcfb68da..6e3e6f2d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java @@ -19,66 +19,64 @@ package org.apache.flink.streaming.connectors.elasticsearch.table; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; -import org.junit.Test; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.Collections; +import java.util.List; import java.util.function.Function; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.nullValue; -import static org.junit.Assert.assertThat; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** Tests for {@link KeyExtractor}. */ public class KeyExtractorTest { @Test public void testSimpleKey() { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .primaryKey("a") - .build(); + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType())) + .collect(Collectors.toList()); - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); - assertThat(key, equalTo("12")); + Assertions.assertEquals(key, "12"); } @Test public void testNoPrimaryKey() { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .build(); + List logicalTypesWithIndex = Collections.emptyList(); - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); - assertThat(key, nullValue()); + Assertions.assertEquals(key, null); } @Test public void testTwoFieldsKey() { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.BIGINT().notNull()) - .field("b", DataTypes.STRING()) - .field("c", DataTypes.TIMESTAMP().notNull()) - .primaryKey("a", "c") - .build(); + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.TIMESTAMP().notNull().getLogicalType())) + .collect(Collectors.toList()); - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); String key = keyExtractor.apply( @@ -87,29 +85,44 @@ public void testTwoFieldsKey() { StringData.fromString("ABCD"), TimestampData.fromLocalDateTime( LocalDateTime.parse("2012-12-12T12:12:12")))); - assertThat(key, equalTo("12_2012-12-12T12:12:12")); + Assertions.assertEquals(key, "12_2012-12-12T12:12:12"); } @Test public void testAllTypesKey() { - TableSchema schema = - TableSchema.builder() - .field("a", DataTypes.TINYINT().notNull()) - .field("b", DataTypes.SMALLINT().notNull()) - .field("c", DataTypes.INT().notNull()) - .field("d", DataTypes.BIGINT().notNull()) - .field("e", DataTypes.BOOLEAN().notNull()) - .field("f", DataTypes.FLOAT().notNull()) - .field("g", DataTypes.DOUBLE().notNull()) - .field("h", DataTypes.STRING().notNull()) - .field("i", DataTypes.TIMESTAMP().notNull()) - .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) - .field("k", DataTypes.TIME().notNull()) - .field("l", DataTypes.DATE().notNull()) - .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") - .build(); - - Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + List logicalTypesWithIndex = + Stream.of( + new LogicalTypeWithIndex( + 0, DataTypes.TINYINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 1, DataTypes.SMALLINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 2, DataTypes.INT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 3, DataTypes.BIGINT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 4, DataTypes.BOOLEAN().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 5, DataTypes.FLOAT().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 6, DataTypes.DOUBLE().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 7, DataTypes.STRING().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 8, DataTypes.TIMESTAMP().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 9, + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE() + .notNull() + .getLogicalType()), + new LogicalTypeWithIndex( + 10, DataTypes.TIME().notNull().getLogicalType()), + new LogicalTypeWithIndex( + 11, DataTypes.DATE().notNull().getLogicalType())) + .collect(Collectors.toList()); + + Function keyExtractor = + KeyExtractor.createKeyExtractor(logicalTypesWithIndex, "_"); String key = keyExtractor.apply( @@ -127,9 +140,8 @@ public void testAllTypesKey() { TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), (int) LocalDate.parse("2015-05-15").toEpochDay())); - assertThat( + Assertions.assertEquals( key, - equalTo( - "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15"); } } From a8cfe8479a7985e5e97d7b1b1722ead3275f7b6d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Wed, 3 Nov 2021 15:00:39 +0100 Subject: [PATCH 165/207] [hotfix][connectors/elasticsearch] Fix license formatting --- .../elasticsearch/sink/BulkProcessorConfig.java | 15 ++++++++------- .../elasticsearch/sink/ElasticsearchEmitter.java | 15 ++++++++------- .../elasticsearch/sink/ElasticsearchSink.java | 15 ++++++++------- .../sink/ElasticsearchSinkBuilderBase.java | 15 ++++++++------- .../elasticsearch/sink/ElasticsearchWriter.java | 15 ++++++++------- .../elasticsearch/sink/FlushBackoffType.java | 15 ++++++++------- .../elasticsearch/sink/NetworkClientConfig.java | 15 ++++++++------- .../elasticsearch/sink/RequestIndexer.java | 15 ++++++++------- 8 files changed, 64 insertions(+), 56 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java index 4ebe852a..519a4473 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java index ad368ad5..b3815afe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index b1bf79de..9bf756a0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index 979cab21..87588ced 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 59a4fc85..4663da9b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java index 5a33208f..1e001f04 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java index 647d0213..e093008a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java index 40669e0c..2bb8732f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java @@ -1,12 +1,13 @@ /* - * 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 + * 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 + * 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, From 9eca6befd320bd77d904d0fc876436655c1de0f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Wed, 3 Nov 2021 15:55:09 +0100 Subject: [PATCH 166/207] [hotfix][connectors/elasticsearch] Move files from streaming.connectors.elasticsearch.table to connector.elasticsearch.table --- .../table/AbstractTimeIndexGenerator.java | 2 +- .../table/ElasticsearchConfiguration.java | 28 ++++++++-------- .../table/ElasticsearchConnectorOptions.java | 2 +- .../table/ElasticsearchDynamicSink.java | 2 +- .../ElasticsearchDynamicSinkFactoryBase.java | 32 +++++++++---------- .../ElasticsearchSinkBuilderSupplier.java | 2 +- .../table/ElasticsearchValidationUtils.java | 2 +- .../elasticsearch/table/IndexGenerator.java | 2 +- .../table/IndexGeneratorBase.java | 2 +- .../table/IndexGeneratorFactory.java | 2 +- .../elasticsearch/table/KeyExtractor.java | 2 +- .../table/LogicalTypeWithIndex.java | 2 +- .../table/RowElasticsearchEmitter.java | 2 +- .../table/SerializableFunction.java | 2 +- .../table/StaticIndexGenerator.java | 2 +- .../ElasticsearchDynamicSinkBaseITCase.java | 2 +- ...asticsearchDynamicSinkFactoryBaseTest.java | 2 +- .../table/IndexGeneratorTest.java | 2 +- .../elasticsearch/table/KeyExtractorTest.java | 2 +- .../elasticsearch/table/TestContext.java | 2 +- .../table/Elasticsearch6Configuration.java | 4 +-- .../table/Elasticsearch6ConnectorOptions.java | 2 +- .../Elasticsearch6DynamicSinkFactory.java | 4 +-- .../org.apache.flink.table.factories.Factory | 2 +- .../Elasticsearch6DynamicSinkFactoryTest.java | 4 +-- .../Elasticsearch6DynamicSinkITCase.java | 4 +-- .../Elasticsearch7DynamicSinkFactory.java | 2 +- .../org.apache.flink.table.factories.Factory | 2 +- .../Elasticsearch7DynamicSinkFactoryTest.java | 4 +-- .../Elasticsearch7DynamicSinkITCase.java | 4 +-- 30 files changed, 64 insertions(+), 64 deletions(-) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/AbstractTimeIndexGenerator.java (95%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchConfiguration.java (71%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchConnectorOptions.java (98%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchDynamicSink.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java (82%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java (93%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchValidationUtils.java (98%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/IndexGenerator.java (95%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/IndexGeneratorBase.java (95%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/IndexGeneratorFactory.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/KeyExtractor.java (98%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/LogicalTypeWithIndex.java (94%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/RowElasticsearchEmitter.java (98%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/SerializableFunction.java (93%) rename flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/StaticIndexGenerator.java (94%) rename flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/IndexGeneratorTest.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/KeyExtractorTest.java (99%) rename flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/TestContext.java (97%) rename flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch6Configuration.java (86%) rename flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch6ConnectorOptions.java (95%) rename flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java (93%) rename flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java (92%) rename flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java (95%) rename flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java (95%) rename flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java (92%) rename flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/{streaming/connectors => connector}/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java (95%) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java index 6c22cf3b..2a6dd929 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java similarity index 71% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java index 78a31022..ee6488e4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.MemorySize; @@ -32,19 +32,19 @@ import java.util.Optional; import java.util.stream.Collectors; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; import static org.apache.flink.util.Preconditions.checkNotNull; /** Elasticsearch base configuration. */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java similarity index 98% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java index 404d3ca1..5cff4ee7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index 3c2b6bd3..d432d999 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java similarity index 82% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index 6d545289..2d022136 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; @@ -46,22 +46,22 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.commons.lang3.StringUtils.capitalize; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.elasticsearch.common.Strings.capitalize; /** A {@link DynamicTableSinkFactory} for discovering ElasticsearchDynamicSink. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java similarity index 93% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java index 1c3ce9fa..411453cc 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java similarity index 98% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java index 40d89196..3d60f345 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.ValidationException; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java index f0f0a46b..c34c9b2b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java index adfcaa46..8a89013e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java index 2cd903d3..baf0e04f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableException; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java similarity index 98% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java index 888994d9..01b820cd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableColumn; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java similarity index 94% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java index 81c63475..c43b7f48 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/LogicalTypeWithIndex.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.table.types.logical.LogicalType; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java similarity index 98% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java index 236791c7..8b0fd07a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.connector.sink.SinkWriter; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java similarity index 93% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java index be243134..d120a04d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/SerializableFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import java.io.Serializable; import java.util.function.Function; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java similarity index 94% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java index 1ffcac48..67e0a1fd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index 38a167aa..e256a51a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java index a89e0f96..29e3d212 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java index 7c8db881..de2aad58 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java similarity index 99% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java index 6e3e6f2d..e2110ca7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.data.GenericRowData; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java similarity index 97% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java rename to flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java index a6dba08a..edea4fce 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java similarity index 86% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java rename to flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java index 638f390d..03705509 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java @@ -16,12 +16,12 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ReadableConfig; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; /** Elasticsearch 6 specific configuration. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java rename to flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java index 7620d2ee..d9ff53ca 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6ConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java similarity index 93% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java rename to flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 5f05e608..2bb2c8a5 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigOption; @@ -29,7 +29,7 @@ import java.util.Set; -import static org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.connector.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; /** A {@link DynamicTableSinkFactory} for discovering {@link ElasticsearchDynamicSink}. */ @Internal diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 29a85938..bb5a894b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6DynamicSinkFactory +org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java similarity index 92% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java rename to flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java index 3a384edf..d8053d4c 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.junit.jupiter.api.Test; -import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ public class Elasticsearch6DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java rename to flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 4d2bb5a2..06b45915 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.util.DockerImageVersions; @@ -31,7 +31,7 @@ import java.io.IOException; import java.util.Map; -import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; /** IT tests for {@link ElasticsearchDynamicSink}. */ @Testcontainers diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java rename to flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 0d4b1069..02c8eebd 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; import org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder; diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 10e4846e..9e189e38 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory +org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java similarity index 92% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java rename to flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java index ee217d26..e8247725 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -16,11 +16,11 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.junit.jupiter.api.Test; -import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; /** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ public class Elasticsearch7DynamicSinkFactoryTest extends ElasticsearchDynamicSinkFactoryBaseTest { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java similarity index 95% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java rename to flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 8def2d26..8c6bbb01 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.streaming.connectors.elasticsearch.table; +package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.util.DockerImageVersions; @@ -32,7 +32,7 @@ import java.io.IOException; import java.util.Map; -import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.connector.elasticsearch.table.TestContext.context; /** IT tests for {@link ElasticsearchDynamicSink}. */ @Testcontainers From 8e24b067d1565eddad1495680115e498a606821e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Wed, 10 Nov 2021 17:16:30 +0100 Subject: [PATCH 167/207] [FLINK-24325][connectors/elasticsearch] Make ElasticsearchSinkBuilderBase type-capturing --- .../sink/ElasticsearchSinkBuilderBase.java | 110 ++++++++++++------ .../table/ElasticsearchDynamicSink.java | 3 +- .../ElasticsearchSinkBuilderSupplier.java | 3 +- .../sink/ElasticsearchSinkBaseITCase.java | 4 +- .../ElasticsearchSinkBuilderBaseTest.java | 46 ++++---- .../sink/Elasticsearch6SinkBuilder.java | 33 +++++- .../sink/Elasticsearch6SinkBuilderTest.java | 7 +- .../sink/Elasticsearch6SinkITCase.java | 2 +- .../sink/Elasticsearch7SinkBuilder.java | 32 ++++- .../sink/Elasticsearch7SinkBuilderTest.java | 7 +- .../sink/Elasticsearch7SinkITCase.java | 3 +- 11 files changed, 174 insertions(+), 76 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index 87588ced..51ed46f8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -19,6 +19,8 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.util.InstantiationUtil; @@ -37,7 +39,8 @@ * @param type of the records converted to Elasticsearch actions */ @PublicEvolving -public abstract class ElasticsearchSinkBuilderBase { +public abstract class ElasticsearchSinkBuilderBase< + IN, B extends ElasticsearchSinkBuilderBase> { private int bulkFlushMaxActions = 1000; private int bulkFlushMaxMb = -1; @@ -47,7 +50,7 @@ public abstract class ElasticsearchSinkBuilderBase { private long bulkFlushBackOffDelay = -1; private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; private List hosts; - private ElasticsearchEmitter emitter; + protected ElasticsearchEmitter emitter; private String username; private String password; private String connectionPathPrefix; @@ -55,20 +58,8 @@ public abstract class ElasticsearchSinkBuilderBase { protected ElasticsearchSinkBuilderBase() {} @SuppressWarnings("unchecked") - private ElasticsearchSinkBuilderBase self() { - return (ElasticsearchSinkBuilderBase) this; - } - - /** - * Sets the hosts where the Elasticsearch cluster nodes are reachable. - * - * @param hosts http addresses describing the node locations - */ - public ElasticsearchSinkBuilderBase setHosts(HttpHost... hosts) { - checkNotNull(hosts); - checkState(hosts.length > 0, "Hosts cannot be empty."); - this.hosts = Arrays.asList(hosts); - return this; + protected > S self() { + return (S) this; } /** @@ -77,18 +68,31 @@ public ElasticsearchSinkBuilderBase setHosts(HttpHost... hosts) { * @param emitter to process records into Elasticsearch actions. * @return this builder */ - public ElasticsearchSinkBuilderBase setEmitter( + public ElasticsearchSinkBuilderBase setEmitter( ElasticsearchEmitter emitter) { checkNotNull(emitter); checkState( InstantiationUtil.isSerializable(emitter), "The elasticsearch emitter must be serializable."); - final ElasticsearchSinkBuilderBase self = self(); + final ElasticsearchSinkBuilderBase self = self(); self.emitter = emitter; return self; } + /** + * Sets the hosts where the Elasticsearch cluster nodes are reachable. + * + * @param hosts http addresses describing the node locations + * @return this builder + */ + public B setHosts(HttpHost... hosts) { + checkNotNull(hosts); + checkState(hosts.length > 0, "Hosts cannot be empty."); + this.hosts = Arrays.asList(hosts); + return self(); + } + /** * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link * DeliveryGuarantee#NONE} @@ -96,13 +100,12 @@ public ElasticsearchSinkBuilderBase setEmitter( * @param deliveryGuarantee which describes the record emission behaviour * @return this builder */ - public ElasticsearchSinkBuilderBase setDeliveryGuarantee( - DeliveryGuarantee deliveryGuarantee) { + public B setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) { checkState( deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE, "Elasticsearch sink does not support the EXACTLY_ONCE guarantee."); this.deliveryGuarantee = checkNotNull(deliveryGuarantee); - return this; + return self(); } /** @@ -112,12 +115,12 @@ public ElasticsearchSinkBuilderBase setDeliveryGuarantee( * @param numMaxActions the maximum number of actions to buffer per bulk request. * @return this builder */ - public ElasticsearchSinkBuilderBase setBulkFlushMaxActions(int numMaxActions) { + public B setBulkFlushMaxActions(int numMaxActions) { checkState( numMaxActions == -1 || numMaxActions > 0, "Max number of buffered actions must be larger than 0."); this.bulkFlushMaxActions = numMaxActions; - return this; + return self(); } /** @@ -127,12 +130,12 @@ public ElasticsearchSinkBuilderBase setBulkFlushMaxActions(int numMaxActions * @param maxSizeMb the maximum size of buffered actions, in mb. * @return this builder */ - public ElasticsearchSinkBuilderBase setBulkFlushMaxSizeMb(int maxSizeMb) { + public B setBulkFlushMaxSizeMb(int maxSizeMb) { checkState( maxSizeMb == -1 || maxSizeMb > 0, "Max size of buffered actions must be larger than 0."); this.bulkFlushMaxMb = maxSizeMb; - return this; + return self(); } /** @@ -141,13 +144,13 @@ public ElasticsearchSinkBuilderBase setBulkFlushMaxSizeMb(int maxSizeMb) { * @param intervalMillis the bulk flush interval, in milliseconds. * @return this builder */ - public ElasticsearchSinkBuilderBase setBulkFlushInterval(long intervalMillis) { + public B setBulkFlushInterval(long intervalMillis) { checkState( intervalMillis == -1 || intervalMillis >= 0, "Interval (in milliseconds) between each flush must be larger than " + "or equal to 0."); this.bulkFlushInterval = intervalMillis; - return this; + return self(); } /** @@ -162,7 +165,7 @@ public ElasticsearchSinkBuilderBase setBulkFlushInterval(long intervalMillis * @param flushBackoffType the backoff type to use. * @return this builder */ - public ElasticsearchSinkBuilderBase setBulkFlushBackoffStrategy( + public B setBulkFlushBackoffStrategy( FlushBackoffType flushBackoffType, int maxRetries, long delayMillis) { this.bulkFlushBackoffType = checkNotNull(flushBackoffType); checkState( @@ -175,7 +178,7 @@ public ElasticsearchSinkBuilderBase setBulkFlushBackoffStrategy( "Delay (in milliseconds) between each backoff attempt must be larger " + "than or equal to 0."); this.bulkFlushBackOffDelay = delayMillis; - return this; + return self(); } /** @@ -184,10 +187,10 @@ public ElasticsearchSinkBuilderBase setBulkFlushBackoffStrategy( * @param username of the Elasticsearch cluster user * @return this builder */ - public ElasticsearchSinkBuilderBase setConnectionUsername(String username) { + public B setConnectionUsername(String username) { checkNotNull(username); this.username = username; - return this; + return self(); } /** @@ -196,10 +199,10 @@ public ElasticsearchSinkBuilderBase setConnectionUsername(String username) { * @param password of the Elasticsearch cluster user * @return this builder */ - public ElasticsearchSinkBuilderBase setConnectionPassword(String password) { + public B setConnectionPassword(String password) { checkNotNull(password); this.password = password; - return this; + return self(); } /** @@ -208,13 +211,13 @@ public ElasticsearchSinkBuilderBase setConnectionPassword(String password) { * @param prefix for the communication * @return this builder */ - public ElasticsearchSinkBuilderBase setConnectionPathPrefix(String prefix) { + public B setConnectionPathPrefix(String prefix) { checkNotNull(prefix); this.connectionPathPrefix = prefix; - return this; + return self(); } - abstract BulkRequestConsumerFactory getBulkRequestConsumer(); + protected abstract BulkRequestConsumerFactory getBulkRequestConsumer(); /** * Constructs the {@link ElasticsearchSink} with the properties configured this builder. @@ -240,6 +243,8 @@ private NetworkClientConfig buildNetworkClientConfig() { private BulkProcessorConfig buildBulkProcessorConfig() { BulkRequestConsumerFactory bulkRequestConsumer = getBulkRequestConsumer(); + ClosureCleaner.clean( + bulkRequestConsumer, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); checkNotNull(bulkRequestConsumer); return new BulkProcessorConfig( @@ -251,4 +256,37 @@ private BulkProcessorConfig buildBulkProcessorConfig() { bulkFlushBackOffDelay, bulkRequestConsumer); } + + @Override + public String toString() { + return "ElasticsearchSinkBuilder{" + + "bulkFlushMaxActions=" + + bulkFlushMaxActions + + ", bulkFlushMaxMb=" + + bulkFlushMaxMb + + ", bulkFlushInterval=" + + bulkFlushInterval + + ", bulkFlushBackoffType=" + + bulkFlushBackoffType + + ", bulkFlushBackoffRetries=" + + bulkFlushBackoffRetries + + ", bulkFlushBackOffDelay=" + + bulkFlushBackOffDelay + + ", deliveryGuarantee=" + + deliveryGuarantee + + ", hosts=" + + hosts + + ", emitter=" + + emitter + + ", username='" + + username + + '\'' + + ", password='" + + password + + '\'' + + ", connectionPathPrefix='" + + connectionPathPrefix + + '\'' + + '}'; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index d432d999..e5f94f19 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -112,7 +112,8 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { documentType, createKeyExtractor()); - final ElasticsearchSinkBuilderBase builder = builderSupplier.get(); + ElasticsearchSinkBuilderBase builder = + builderSupplier.get(); builder.setEmitter(rowElasticsearchEmitter); builder.setHosts(config.getHosts().toArray(new HttpHost[0])); builder.setDeliveryGuarantee(config.getDeliveryGuarantee()); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java index 411453cc..70088021 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java @@ -22,4 +22,5 @@ import java.util.function.Supplier; -interface ElasticsearchSinkBuilderSupplier extends Supplier> {} +interface ElasticsearchSinkBuilderSupplier + extends Supplier> {} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java index b1fbd8c1..757bf311 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java @@ -75,7 +75,9 @@ abstract class ElasticsearchSinkBaseITCase { abstract TestClientBase createTestClient(RestHighLevelClient client); - abstract ElasticsearchSinkBuilderBase> getSinkBuilder(); + abstract ElasticsearchSinkBuilderBase< + Tuple2, ? extends ElasticsearchSinkBuilderBase> + getSinkBuilder(); private RestHighLevelClient createRestHighLevelClient() { final CredentialsProvider credentialsProvider = new BasicCredentialsProvider(); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index 270b15b1..e65d844a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -20,28 +20,40 @@ import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; - import org.apache.http.HttpHost; +import org.junit.jupiter.api.DynamicTest; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestFactory; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.MethodSource; -import java.util.List; +import java.util.stream.Stream; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; /** Tests for {@link ElasticsearchSinkBuilderBase}. */ @ExtendWith(TestLoggerExtension.class) @TestInstance(TestInstance.Lifecycle.PER_CLASS) -abstract class ElasticsearchSinkBuilderBaseTest { +abstract class ElasticsearchSinkBuilderBaseTest> { + + @TestFactory + Stream testValidBuilders() { + Stream validBuilders = + Stream.of( + createMinimalBuilder(), + createMinimalBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), + createMinimalBuilder() + .setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), + createMinimalBuilder() + .setConnectionUsername("username") + .setConnectionPassword("password")); - @ParameterizedTest - @MethodSource("validBuilders") - void testBuildElasticsearchSink(ElasticsearchSinkBuilderBase builder) { - builder.build(); + return DynamicTest.stream( + validBuilders, + ElasticsearchSinkBuilderBase::toString, + builder -> assertDoesNotThrow(builder::build)); } @Test @@ -65,17 +77,7 @@ void testThrowIfEmitterNotSet() { () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build()); } - private List> validBuilders() { - return Lists.newArrayList( - createMinimalBuilder(), - createMinimalBuilder().setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE), - createMinimalBuilder().setBulkFlushBackoffStrategy(FlushBackoffType.CONSTANT, 1, 1), - createMinimalBuilder() - .setConnectionUsername("username") - .setConnectionPassword("password")); - } - - abstract ElasticsearchSinkBuilderBase createEmptyBuilder(); + abstract B createEmptyBuilder(); - abstract ElasticsearchSinkBuilderBase createMinimalBuilder(); + abstract B createMinimalBuilder(); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java index e766cb11..ea30e39f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -20,6 +20,11 @@ import org.apache.flink.annotation.PublicEvolving; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.RestHighLevelClient; + /** * Builder to construct an Elasticsearch 6 compatible {@link ElasticsearchSink}. * @@ -43,12 +48,34 @@ * @param type of the records converted to Elasticsearch actions */ @PublicEvolving -public class Elasticsearch6SinkBuilder extends ElasticsearchSinkBuilderBase { +public class Elasticsearch6SinkBuilder + extends ElasticsearchSinkBuilderBase> { public Elasticsearch6SinkBuilder() {} @Override - public BulkRequestConsumerFactory getBulkRequestConsumer() { - return client -> client::bulkAsync; + public Elasticsearch6SinkBuilder setEmitter( + ElasticsearchEmitter emitter) { + super.setEmitter(emitter); + return self(); + } + + @Override + protected BulkRequestConsumerFactory getBulkRequestConsumer() { + return new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda because the + // deserialization fails then + @Override + public BulkRequestFactory create(RestHighLevelClient client) { + return new BulkRequestFactory() { // This cannot be inlined as a lambda because the + // deserialization fails then + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener bulkResponseActionListener) { + client.bulkAsync(bulkRequest, bulkResponseActionListener); + } + }; + } + }; } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java index 0372caa0..3de02f50 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java @@ -20,15 +20,16 @@ import org.apache.http.HttpHost; /** Tests for {@link Elasticsearch6SinkBuilder}. */ -class Elasticsearch6SinkBuilderTest extends ElasticsearchSinkBuilderBaseTest { +class Elasticsearch6SinkBuilderTest + extends ElasticsearchSinkBuilderBaseTest> { @Override - ElasticsearchSinkBuilderBase createEmptyBuilder() { + Elasticsearch6SinkBuilder createEmptyBuilder() { return new Elasticsearch6SinkBuilder<>(); } @Override - ElasticsearchSinkBuilderBase createMinimalBuilder() { + Elasticsearch6SinkBuilder createMinimalBuilder() { return new Elasticsearch6SinkBuilder<>() .setEmitter((element, indexer, context) -> {}) .setHosts(new HttpHost("localhost:3000")); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java index 3df6c9af..6aa743f5 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java @@ -49,7 +49,7 @@ TestClientBase createTestClient(RestHighLevelClient client) { } @Override - ElasticsearchSinkBuilderBase> getSinkBuilder() { + Elasticsearch6SinkBuilder> getSinkBuilder() { return new Elasticsearch6SinkBuilder<>(); } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java index 30986dc0..b244b79f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -19,7 +19,11 @@ import org.apache.flink.annotation.PublicEvolving; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; /** * Builder to construct an Elasticsearch 7 compatible {@link ElasticsearchSink}. @@ -44,15 +48,35 @@ * @param type of the records converted to Elasticsearch actions */ @PublicEvolving -public class Elasticsearch7SinkBuilder extends ElasticsearchSinkBuilderBase { +public class Elasticsearch7SinkBuilder + extends ElasticsearchSinkBuilderBase> { public Elasticsearch7SinkBuilder() {} @Override - public BulkRequestConsumerFactory getBulkRequestConsumer() { - return (client) -> - (bulkRequest, bulkResponseActionListener) -> + public Elasticsearch7SinkBuilder setEmitter( + ElasticsearchEmitter emitter) { + super.setEmitter(emitter); + return self(); + } + + @Override + protected BulkRequestConsumerFactory getBulkRequestConsumer() { + return new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda because the + // deserialization fails then + @Override + public BulkRequestFactory create(RestHighLevelClient client) { + return new BulkRequestFactory() { // This cannot be inlined as a lambda because the + // deserialization fails then + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener bulkResponseActionListener) { client.bulkAsync( bulkRequest, RequestOptions.DEFAULT, bulkResponseActionListener); + } + }; + } + }; } } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java index 052dd8ad..0d6250ea 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java @@ -20,15 +20,16 @@ import org.apache.http.HttpHost; /** Tests for {@link Elasticsearch7SinkBuilder}. */ -class Elasticsearch7SinkBuilderTest extends ElasticsearchSinkBuilderBaseTest { +class Elasticsearch7SinkBuilderTest + extends ElasticsearchSinkBuilderBaseTest> { @Override - ElasticsearchSinkBuilderBase createEmptyBuilder() { + Elasticsearch7SinkBuilder createEmptyBuilder() { return new Elasticsearch7SinkBuilder<>(); } @Override - ElasticsearchSinkBuilderBase createMinimalBuilder() { + Elasticsearch7SinkBuilder createMinimalBuilder() { return new Elasticsearch7SinkBuilder<>() .setEmitter((element, indexer, context) -> {}) .setHosts(new HttpHost("localhost:3000")); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java index 25596df7..b0cd52dd 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java @@ -49,7 +49,8 @@ TestClientBase createTestClient(RestHighLevelClient client) { } @Override - ElasticsearchSinkBuilderBase> getSinkBuilder() { + ElasticsearchSinkBuilderBase, ? extends ElasticsearchSinkBuilderBase> + getSinkBuilder() { return new Elasticsearch7SinkBuilder<>(); } } From d540ea314db9459c20ad17ceaa8132fe9b74c6e9 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Fri, 26 Nov 2021 11:42:36 +0100 Subject: [PATCH 168/207] [FLINK-24596][core] Introduce SerializableFunction and unify usages --- .../elasticsearch/table/KeyExtractor.java | 1 + .../table/SerializableFunction.java | 24 ------------------- 2 files changed, 1 insertion(+), 24 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java index 01b820cd..6beff6d7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java @@ -23,6 +23,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.function.SerializableFunction; import java.io.Serializable; import java.time.Duration; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java deleted file mode 100644 index d120a04d..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/SerializableFunction.java +++ /dev/null @@ -1,24 +0,0 @@ -/* - * 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.connector.elasticsearch.table; - -import java.io.Serializable; -import java.util.function.Function; - -interface SerializableFunction extends Function, Serializable {} From f05f6bc4311eda9d62e4d3b0c19c19b41652ddb9 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 25 Nov 2021 10:49:38 +0100 Subject: [PATCH 169/207] [FLINK-25060][table-common] Replace projection methods of FLINK-24399 with the new Projection util This closes #17906. --- .../table/ElasticsearchDynamicSinkFactoryBase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index 2d022136..e124ad6b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.Projection; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; @@ -170,7 +171,7 @@ List getPrimaryKeyLogicalTypesWithIndex(Context context) { DataType physicalRowDataType = context.getPhysicalRowDataType(); int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); if (primaryKeyIndexes.length != 0) { - DataType pkDataType = DataType.projectFields(physicalRowDataType, primaryKeyIndexes); + DataType pkDataType = Projection.of(primaryKeyIndexes).project(physicalRowDataType); ElasticsearchValidationUtils.validatePrimaryKey(pkDataType); } From c151b2a90fc4d2e31f9cb6777b12a2499e4a6c38 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Wed, 24 Nov 2021 15:13:07 +0100 Subject: [PATCH 170/207] [FLINK-24687][table][connectors] Move FileSystemTableSink, FileSystemTableSource to flink-connector-files and columnar support to flink-table-common Now table packages don't depend on flink-connector-files anymore. Fix orc and parquet format to use only common classes and not planner nor runtime classes. - [connector-files] Add @Internal to all public classes and interfaces - [orc][parquet][hive] Drop scala suffix from flink-orc and flink-parquet - [architecture-tests] Updated the violations file - [connector-elasticsearch-base] Add flink-connector-base as dependency, which was previously brought in through flink-table-api-java-bridge -> flink-table-api-java -> flink-table-common -> flink-connector-files -> flink-connector-base. - [orc][parquet] Add issue link for partition keys handling - [table-uber][dist] Now flink-connector-files is not shaded inside table-uber anymore but it's loaded in /lib in the distribution as flink-connector-files - [docs] Update sql_connectors.yml This closes #17897. --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index bff4b096..612048b1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -44,6 +44,11 @@ under the License. + + org.apache.flink + flink-connector-base + ${project.version} + org.apache.flink flink-streaming-java From ae67acfd26e14bcbb97f0cbb03cc89286019ab17 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 7 Dec 2021 10:35:11 +0100 Subject: [PATCH 171/207] [FLINK-24372][connectors/elasticsearch] Deprecate old (pre FLIP-143) Elasticsearch connector --- .../elasticsearch/ActionRequestFailureHandler.java | 3 +++ .../connectors/elasticsearch/ElasticsearchSinkFunction.java | 2 ++ .../streaming/connectors/elasticsearch/RequestIndexer.java | 3 +++ .../util/RetryRejectedExecutionFailureHandler.java | 3 +++ .../connectors/elasticsearch6/ElasticsearchSink.java | 6 ++++++ .../connectors/elasticsearch6/RestClientFactory.java | 3 +++ .../connectors/elasticsearch7/ElasticsearchSink.java | 6 ++++++ .../connectors/elasticsearch7/RestClientFactory.java | 3 +++ 8 files changed, 29 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java index a13d10a9..8a3fed98 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java @@ -59,7 +59,10 @@ * exact type could not be retrieved through the older version Java client APIs (thus, the types * will be general {@link Exception}s and only differ in the failure message). In this case, it is * recommended to match on the provided REST status code. + * + * @deprecated This has been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public interface ActionRequestFailureHandler extends Serializable { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java index dfbeec84..0cf85239 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java @@ -56,7 +56,9 @@ * } * * @param The type of the element handled by this {@code ElasticsearchSinkFunction} + * @deprecated This has been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public interface ElasticsearchSinkFunction extends Serializable, Function { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index 3cf748f6..747c690a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -28,7 +28,10 @@ /** * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them * for sending to an Elasticsearch cluster. + * + * @deprecated This has been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public interface RequestIndexer { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java index 7b0b861d..bfa7b8ca 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java @@ -32,7 +32,10 @@ * An {@link ActionRequestFailureHandler} that re-adds requests that failed due to temporary {@link * EsRejectedExecutionException}s (which means that Elasticsearch node queues are currently full), * and fails for all other failures. + * + * @deprecated This hase been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public class RetryRejectedExecutionFailureHandler implements ActionRequestFailureHandler { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 86fb7744..22b5051d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -58,7 +58,10 @@ * of {@link ElasticsearchSinkFunction} for an example. * * @param Type of the elements handled by this sink + * @deprecated This sink has been deprecated in favor of {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} */ +@Deprecated @PublicEvolving public class ElasticsearchSink extends ElasticsearchSinkBase { @@ -82,7 +85,10 @@ private ElasticsearchSink( * A builder for creating an {@link ElasticsearchSink}. * * @param Type of the elements handled by the sink this builder creates. + * @deprecated This has been deprecated, please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder}. */ + @Deprecated @PublicEvolving public static class Builder { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java index 67246bb6..d11eb970 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java @@ -26,7 +26,10 @@ /** * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} * internally used in the {@link ElasticsearchSink}. + * + * @deprecated This has been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public interface RestClientFactory extends Serializable { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java index 92b17d04..4a5de6a1 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java @@ -58,7 +58,10 @@ * of {@link ElasticsearchSinkFunction} for an example. * * @param Type of the elements handled by this sink + * @deprecated This sink has been deprecated in favor of {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} */ +@Deprecated @PublicEvolving public class ElasticsearchSink extends ElasticsearchSinkBase { @@ -82,7 +85,10 @@ private ElasticsearchSink( * A builder for creating an {@link ElasticsearchSink}. * * @param Type of the elements handled by the sink this builder creates. + * @deprecated This has been deprecated, please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkBuilder}. */ + @Deprecated @PublicEvolving public static class Builder { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java index 50e2dfc1..9e056712 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java @@ -26,7 +26,10 @@ /** * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} * internally used in the {@link ElasticsearchSink}. + * + * @deprecated This has been deprecated and will be removed in the future. */ +@Deprecated @PublicEvolving public interface RestClientFactory extends Serializable { From 241977cad3ad6598759f735f5f003ef1863a2035 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Wed, 1 Dec 2021 11:24:03 +0100 Subject: [PATCH 172/207] [FLINK-25189][connectors/elasticsearch] Bump Elasticsearch 7 to v7.15.2 --- .../pom.xml | 2 +- .../sink/BulkProcessorBuilderFactory.java | 36 ++++++ .../sink/BulkProcessorConfig.java | 9 +- .../sink/BulkRequestConsumerFactory.java | 10 +- .../elasticsearch/sink/ElasticsearchSink.java | 4 + .../sink/ElasticsearchSinkBuilderBase.java | 21 ++-- .../sink/ElasticsearchWriter.java | 56 ++------- .../ElasticsearchApiCallBridge.java | 10 ++ .../elasticsearch/ElasticsearchSinkBase.java | 5 +- .../sink/ElasticsearchWriterITCase.java | 110 ++++++++++++------ .../ElasticsearchSinkBaseTest.java | 10 +- .../testutils/ElasticsearchResource.java | 2 +- .../sink/Elasticsearch6SinkBuilder.java | 77 +++++++++--- .../Elasticsearch6ApiCallBridge.java | 6 + .../flink-connector-elasticsearch7/pom.xml | 18 ++- .../sink/Elasticsearch7SinkBuilder.java | 80 ++++++++++--- .../Elasticsearch7ApiCallBridge.java | 8 +- .../sink/Elasticsearch7SinkITCase.java | 3 +- .../pom.xml | 6 + .../src/main/resources/META-INF/NOTICE | 58 ++++----- 20 files changed, 357 insertions(+), 174 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 612048b1..e5c30d9a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 7.5.1 + 7.15.2 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java new file mode 100644 index 00000000..f4a76989 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java @@ -0,0 +1,36 @@ +/* + * 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.connector.elasticsearch.sink; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.function.TriFunction; + +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.io.Serializable; + +@Internal +interface BulkProcessorBuilderFactory + extends Serializable, + TriFunction< + RestHighLevelClient, + BulkProcessorConfig, + BulkProcessor.Listener, + BulkProcessor.Builder> {} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java index 519a4473..35fa1ecb 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java @@ -30,7 +30,6 @@ class BulkProcessorConfig implements Serializable { private final FlushBackoffType flushBackoffType; private final int bulkFlushBackoffRetries; private final long bulkFlushBackOffDelay; - private final BulkRequestConsumerFactory bulkRequestConsumerFactory; BulkProcessorConfig( int bulkFlushMaxActions, @@ -38,15 +37,13 @@ class BulkProcessorConfig implements Serializable { long bulkFlushInterval, FlushBackoffType flushBackoffType, int bulkFlushBackoffRetries, - long bulkFlushBackOffDelay, - BulkRequestConsumerFactory bulkRequestConsumerFactory) { + long bulkFlushBackOffDelay) { this.bulkFlushMaxActions = bulkFlushMaxActions; this.bulkFlushMaxMb = bulkFlushMaxMb; this.bulkFlushInterval = bulkFlushInterval; this.flushBackoffType = checkNotNull(flushBackoffType); this.bulkFlushBackoffRetries = bulkFlushBackoffRetries; this.bulkFlushBackOffDelay = bulkFlushBackOffDelay; - this.bulkRequestConsumerFactory = bulkRequestConsumerFactory; } public int getBulkFlushMaxActions() { @@ -72,8 +69,4 @@ public int getBulkFlushBackoffRetries() { public long getBulkFlushBackOffDelay() { return bulkFlushBackOffDelay; } - - public BulkRequestConsumerFactory getBulkRequestConsumerFactory() { - return bulkRequestConsumerFactory; - } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java index e8b45319..7ef9eab2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java @@ -23,9 +23,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.client.RestHighLevelClient; -import java.io.Serializable; import java.util.function.BiConsumer; /** @@ -33,9 +31,5 @@ * across different Elasticsearch versions. */ @Internal -interface BulkRequestConsumerFactory extends Serializable { - BulkRequestFactory create(RestHighLevelClient client); - - interface BulkRequestFactory - extends Serializable, BiConsumer> {} -} +interface BulkRequestConsumerFactory + extends BiConsumer> {} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index 9bf756a0..4a47fe6d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -60,6 +60,7 @@ public class ElasticsearchSink implements Sink { private final List hosts; private final ElasticsearchEmitter emitter; private final BulkProcessorConfig buildBulkProcessorConfig; + private final BulkProcessorBuilderFactory bulkProcessorBuilderFactory; private final NetworkClientConfig networkClientConfig; private final DeliveryGuarantee deliveryGuarantee; @@ -67,9 +68,11 @@ public class ElasticsearchSink implements Sink { List hosts, ElasticsearchEmitter emitter, DeliveryGuarantee deliveryGuarantee, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, BulkProcessorConfig buildBulkProcessorConfig, NetworkClientConfig networkClientConfig) { this.hosts = checkNotNull(hosts); + this.bulkProcessorBuilderFactory = checkNotNull(bulkProcessorBuilderFactory); checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); this.emitter = checkNotNull(emitter); this.deliveryGuarantee = checkNotNull(deliveryGuarantee); @@ -85,6 +88,7 @@ public SinkWriter createWriter(InitContext context, List s emitter, deliveryGuarantee == DeliveryGuarantee.AT_LEAST_ONCE, buildBulkProcessorConfig, + bulkProcessorBuilderFactory, networkClientConfig, context.metricGroup(), context.getMailboxExecutor()); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index 51ed46f8..f187cb6d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -217,7 +217,7 @@ public B setConnectionPathPrefix(String prefix) { return self(); } - protected abstract BulkRequestConsumerFactory getBulkRequestConsumer(); + protected abstract BulkProcessorBuilderFactory getBulkProcessorBuilderFactory(); /** * Constructs the {@link ElasticsearchSink} with the properties configured this builder. @@ -231,8 +231,17 @@ public ElasticsearchSink build() { NetworkClientConfig networkClientConfig = buildNetworkClientConfig(); BulkProcessorConfig bulkProcessorConfig = buildBulkProcessorConfig(); + BulkProcessorBuilderFactory bulkProcessorBuilderFactory = getBulkProcessorBuilderFactory(); + ClosureCleaner.clean( + bulkProcessorBuilderFactory, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); + return new ElasticsearchSink<>( - hosts, emitter, deliveryGuarantee, bulkProcessorConfig, networkClientConfig); + hosts, + emitter, + deliveryGuarantee, + bulkProcessorBuilderFactory, + bulkProcessorConfig, + networkClientConfig); } private NetworkClientConfig buildNetworkClientConfig() { @@ -242,19 +251,13 @@ private NetworkClientConfig buildNetworkClientConfig() { } private BulkProcessorConfig buildBulkProcessorConfig() { - BulkRequestConsumerFactory bulkRequestConsumer = getBulkRequestConsumer(); - ClosureCleaner.clean( - bulkRequestConsumer, ExecutionConfig.ClosureCleanerLevel.RECURSIVE, true); - checkNotNull(bulkRequestConsumer); - return new BulkProcessorConfig( bulkFlushMaxActions, bulkFlushMaxMb, bulkFlushInterval, bulkFlushBackoffType, bulkFlushBackoffRetries, - bulkFlushBackOffDelay, - bulkRequestConsumer); + bulkFlushBackOffDelay); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 4663da9b..f984312b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -32,7 +32,6 @@ import org.apache.http.client.CredentialsProvider; import org.apache.http.impl.client.BasicCredentialsProvider; import org.elasticsearch.action.DocWriteRequest; -import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; @@ -43,9 +42,6 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.RestStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,6 +80,7 @@ class ElasticsearchWriter implements SinkWriter { * checkpoint * @param bulkProcessorConfig describing the flushing and failure handling of the used {@link * BulkProcessor} + * @param bulkProcessorBuilderFactory configuring the {@link BulkProcessor}'s builder * @param networkClientConfig describing properties of the network connection used to connect to * the elasticsearch cluster * @param metricGroup for the sink writer @@ -94,6 +91,7 @@ class ElasticsearchWriter implements SinkWriter { ElasticsearchEmitter emitter, boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig, + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, NetworkClientConfig networkClientConfig, SinkWriterMetricGroup metricGroup, MailboxExecutor mailboxExecutor) { @@ -105,12 +103,7 @@ class ElasticsearchWriter implements SinkWriter { configureRestClientBuilder( RestClient.builder(hosts.toArray(new HttpHost[0])), networkClientConfig)); - this.bulkProcessor = - configureBulkProcessor( - BulkProcessor.builder( - bulkProcessorConfig.getBulkRequestConsumerFactory().create(client), - new BulkListener()), - bulkProcessorConfig); + this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); this.requestIndexer = new DefaultRequestIndexer(); checkNotNull(metricGroup); metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); @@ -179,44 +172,15 @@ private static RestClientBuilder configureRestClientBuilder( return builder; } - private static BulkProcessor configureBulkProcessor( - BulkProcessor.Builder builder, BulkProcessorConfig bulkProcessorConfig) { - // This makes flush() blocking - builder.setConcurrentRequests(0); - - if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { - builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); - } - - if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { - builder.setBulkSize( - new ByteSizeValue(bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); - } + private BulkProcessor createBulkProcessor( + BulkProcessorBuilderFactory bulkProcessorBuilderFactory, + BulkProcessorConfig bulkProcessorConfig) { - if (bulkProcessorConfig.getBulkFlushInterval() != -1) { - builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); - } + BulkProcessor.Builder builder = + bulkProcessorBuilderFactory.apply(client, bulkProcessorConfig, new BulkListener()); - BackoffPolicy backoffPolicy; - final TimeValue backoffDelay = - new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); - final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); - switch (bulkProcessorConfig.getFlushBackoffType()) { - case CONSTANT: - backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); - break; - case EXPONENTIAL: - backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); - break; - case NONE: - backoffPolicy = BackoffPolicy.noBackoff(); - break; - default: - throw new IllegalArgumentException( - "Received unknown backoff policy type " - + bulkProcessorConfig.getFlushBackoffType()); - } - builder.setBackoffPolicy(backoffPolicy); + // This makes flush() blocking + builder.setConcurrentRequests(0); return builder.build(); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 79ab9052..9b4f9fb2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -73,6 +73,16 @@ public interface ElasticsearchApiCallBridge extends Ser @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + /** + * Sets the bulk flush interval, in milliseconds on the provided {@link BulkProcessor.Builder}. + * The builder will be later on used to instantiate the actual {@link BulkProcessor}. + * + * @param builder the {@link BulkProcessor.Builder} to configure. + * @param flushIntervalMillis the flush interval in milliseconds. + */ + void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis); + /** * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. The builder * will be later on used to instantiate the actual {@link BulkProcessor}. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 625adca5..9ed83090 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -37,7 +37,6 @@ import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.rest.RestStatus; import java.io.Serializable; @@ -416,8 +415,8 @@ private void configureFlushInterval(BulkProcessor.Builder bulkProcessorBuilder) if (bulkProcessorFlushIntervalMillis == -1) { bulkProcessorBuilder.setFlushInterval(null); } else { - bulkProcessorBuilder.setFlushInterval( - TimeValue.timeValueMillis(bulkProcessorFlushIntervalMillis)); + callBridge.configureBulkProcessorFlushInterval( + bulkProcessorBuilder, bulkProcessorFlushIntervalMillis); } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 2d98157f..2fa75023 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -32,11 +32,19 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.TimeValue; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -75,14 +83,6 @@ class ElasticsearchWriterITCase { private TestClientBase context; private MetricListener metricListener; - private final BulkRequestConsumerFactory bulkRequestConsumerFactory = - (client) -> - (bulkRequest, bulkResponseActionListener) -> - client.bulkAsync( - bulkRequest, - RequestOptions.DEFAULT, - bulkResponseActionListener); - @BeforeEach void setUp() { metricListener = new MetricListener(); @@ -104,14 +104,7 @@ void testWriteOnBulkFlush() throws Exception { final String index = "test-bulk-flush-without-checkpoint"; final int flushAfterNActions = 5; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig( - flushAfterNActions, - -1, - -1, - FlushBackoffType.NONE, - 0, - 0, - bulkRequestConsumerFactory); + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -144,8 +137,7 @@ void testWriteOnBulkIntervalFlush() throws Exception { // Configure bulk processor to flush every 1s; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig( - -1, -1, 1000, FlushBackoffType.NONE, 0, 0, bulkRequestConsumerFactory); + new BulkProcessorConfig(-1, -1, 1000, FlushBackoffType.NONE, 0, 0); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -163,8 +155,7 @@ void testWriteOnBulkIntervalFlush() throws Exception { void testWriteOnCheckpoint() throws Exception { final String index = "test-bulk-flush-with-checkpoint"; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig( - -1, -1, -1, FlushBackoffType.NONE, 0, 0, bulkRequestConsumerFactory); + new BulkProcessorConfig(-1, -1, -1, FlushBackoffType.NONE, 0, 0); // Enable flush on checkpoint try (final ElasticsearchWriter> writer = @@ -192,14 +183,7 @@ void testIncrementByteOutMetric() throws Exception { metricListener.getMetricGroup(), operatorIOMetricGroup); final int flushAfterNActions = 2; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig( - flushAfterNActions, - -1, - -1, - FlushBackoffType.NONE, - 0, - 0, - bulkRequestConsumerFactory); + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig, metricGroup)) { @@ -226,14 +210,7 @@ void testCurrentSendTime() throws Exception { final String index = "test-current-send-time"; final int flushAfterNActions = 2; final BulkProcessorConfig bulkProcessorConfig = - new BulkProcessorConfig( - flushAfterNActions, - -1, - -1, - FlushBackoffType.NONE, - 0, - 0, - bulkRequestConsumerFactory); + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); try (final ElasticsearchWriter> writer = createWriter(index, false, bulkProcessorConfig)) { @@ -268,11 +245,72 @@ private ElasticsearchWriter> createWriter( TestEmitter.jsonEmitter(index, context.getDataFieldName()), flushOnCheckpoint, bulkProcessorConfig, + new TestBulkProcessorBuilderFactory(), new NetworkClientConfig(null, null, null), metricGroup, new TestMailbox()); } + private static class TestBulkProcessorBuilderFactory implements BulkProcessorBuilderFactory { + @Override + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda + // because then deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval(new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; + } + } + private static class TestClient extends TestClientBase { TestClient(RestHighLevelClient client) { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 67999916..409c6ec2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -601,14 +601,14 @@ public Object answer(InvocationOnMock invocationOnMock) if (mockItemFailure == null) { // the mock response for the item is success mockResponses[i] = - new BulkItemResponse( + BulkItemResponse.success( i, DocWriteRequest.OpType.INDEX, mock(DocWriteResponse.class)); } else { // the mock response for the item is failure mockResponses[i] = - new BulkItemResponse( + BulkItemResponse.failure( i, DocWriteRequest.OpType.INDEX, new BulkItemResponse.Failure( @@ -666,6 +666,12 @@ public Throwable extractFailureCauseFromBulkItemResponse( } } + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + // no need for this in the test cases here + } + @Override public void configureBulkProcessorBackoff( BulkProcessor.Builder builder, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java index cd11b70d..e5716dcf 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java @@ -27,7 +27,7 @@ import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; import org.elasticsearch.client.ClusterAdminClient; -import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.TimeValue; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java index ea30e39f..34c098b8 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -21,9 +21,14 @@ import org.apache.flink.annotation.PublicEvolving; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; /** * Builder to construct an Elasticsearch 6 compatible {@link ElasticsearchSink}. @@ -61,20 +66,66 @@ public Elasticsearch6SinkBuilder setEmitter( } @Override - protected BulkRequestConsumerFactory getBulkRequestConsumer() { - return new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda because the - // deserialization fails then + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { + return new BulkProcessorBuilderFactory() { @Override - public BulkRequestFactory create(RestHighLevelClient client) { - return new BulkRequestFactory() { // This cannot be inlined as a lambda because the - // deserialization fails then - @Override - public void accept( - BulkRequest bulkRequest, - ActionListener bulkResponseActionListener) { - client.bulkAsync(bulkRequest, bulkResponseActionListener); - } - }; + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a + // lambda because then + // deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener + bulkResponseActionListener) { + client.bulkAsync(bulkRequest, bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval( + new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = + BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; } }; } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 89d1e7d8..bd5d5a6a 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -88,6 +88,12 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } } + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + builder.setFlushInterval(TimeValue.timeValueMillis(flushIntervalMillis)); + } + @Override public void configureBulkProcessorBackoff( BulkProcessor.Builder builder, diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index db0e72d8..138c5d29 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -37,7 +37,7 @@ under the License. - 7.5.1 + 7.15.2 @@ -58,7 +58,7 @@ under the License. flink-connector-elasticsearch-base ${project.version} - + org.elasticsearch elasticsearch @@ -82,6 +82,20 @@ under the License. org.elasticsearch.client elasticsearch-rest-high-level-client ${elasticsearch.version} + + + + org.apache.httpcomponents + httpcore-nio + + + + + + + org.apache.httpcomponents + httpcore-nio + 4.4.12 diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java index b244b79f..e981b942 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -20,10 +20,15 @@ import org.apache.flink.annotation.PublicEvolving; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.TimeValue; /** * Builder to construct an Elasticsearch 7 compatible {@link ElasticsearchSink}. @@ -61,21 +66,68 @@ public Elasticsearch7SinkBuilder setEmitter( } @Override - protected BulkRequestConsumerFactory getBulkRequestConsumer() { - return new BulkRequestConsumerFactory() { // This cannot be inlined as a lambda because the - // deserialization fails then + protected BulkProcessorBuilderFactory getBulkProcessorBuilderFactory() { + return new BulkProcessorBuilderFactory() { @Override - public BulkRequestFactory create(RestHighLevelClient client) { - return new BulkRequestFactory() { // This cannot be inlined as a lambda because the - // deserialization fails then - @Override - public void accept( - BulkRequest bulkRequest, - ActionListener bulkResponseActionListener) { - client.bulkAsync( - bulkRequest, RequestOptions.DEFAULT, bulkResponseActionListener); - } - }; + public BulkProcessor.Builder apply( + RestHighLevelClient client, + BulkProcessorConfig bulkProcessorConfig, + BulkProcessor.Listener listener) { + BulkProcessor.Builder builder = + BulkProcessor.builder( + new BulkRequestConsumerFactory() { // This cannot be inlined as a + // lambda because then + // deserialization fails + @Override + public void accept( + BulkRequest bulkRequest, + ActionListener + bulkResponseActionListener) { + client.bulkAsync( + bulkRequest, + RequestOptions.DEFAULT, + bulkResponseActionListener); + } + }, + listener); + + if (bulkProcessorConfig.getBulkFlushMaxActions() != -1) { + builder.setBulkActions(bulkProcessorConfig.getBulkFlushMaxActions()); + } + + if (bulkProcessorConfig.getBulkFlushMaxMb() != -1) { + builder.setBulkSize( + new ByteSizeValue( + bulkProcessorConfig.getBulkFlushMaxMb(), ByteSizeUnit.MB)); + } + + if (bulkProcessorConfig.getBulkFlushInterval() != -1) { + builder.setFlushInterval( + new TimeValue(bulkProcessorConfig.getBulkFlushInterval())); + } + + BackoffPolicy backoffPolicy; + final TimeValue backoffDelay = + new TimeValue(bulkProcessorConfig.getBulkFlushBackOffDelay()); + final int maxRetryCount = bulkProcessorConfig.getBulkFlushBackoffRetries(); + switch (bulkProcessorConfig.getFlushBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff(backoffDelay, maxRetryCount); + break; + case EXPONENTIAL: + backoffPolicy = + BackoffPolicy.exponentialBackoff(backoffDelay, maxRetryCount); + break; + case NONE: + backoffPolicy = BackoffPolicy.noBackoff(); + break; + default: + throw new IllegalArgumentException( + "Received unknown backoff policy type " + + bulkProcessorConfig.getFlushBackoffType()); + } + builder.setBackoffPolicy(backoffPolicy); + return builder; } }; } diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index 6dff0559..9440097a 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -31,7 +31,7 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.TimeValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,6 +92,12 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } } + @Override + public void configureBulkProcessorFlushInterval( + BulkProcessor.Builder builder, long flushIntervalMillis) { + builder.setFlushInterval(TimeValue.timeValueMillis(flushIntervalMillis)); + } + @Override public void configureBulkProcessorBackoff( BulkProcessor.Builder builder, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java index b0cd52dd..fa4638c2 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java @@ -33,8 +33,7 @@ class Elasticsearch7SinkITCase extends ElasticsearchSinkBaseITCase { @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer( - DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_7)) + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)) .withPassword(ELASTICSEARCH_PASSWORD) .withLogConsumer(new Slf4jLogConsumer(LOG)); diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index a5ba0abc..489e9ea3 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -80,6 +80,8 @@ under the License. modules.txt plugins.txt org/joda/** + + org/elasticsearch/bootstrap/** @@ -142,6 +144,10 @@ under the License. com.github.mustachejava org.apache.flink.elasticsearch7.shaded.com.github.mustachejava + + net.jpountz + org.apache.flink.elasticsearch7.shaded.net.jpountz + diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index daf143bb..fe1ccd52 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -20,31 +20,33 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpclient:4.5.13 - org.apache.httpcomponents:httpcore:4.4.14 - org.apache.httpcomponents:httpcore-nio:4.4.12 -- org.apache.lucene:lucene-analyzers-common:8.3.0 -- org.apache.lucene:lucene-backward-codecs:8.3.0 -- org.apache.lucene:lucene-core:8.3.0 -- org.apache.lucene:lucene-grouping:8.3.0 -- org.apache.lucene:lucene-highlighter:8.3.0 -- org.apache.lucene:lucene-join:8.3.0 -- org.apache.lucene:lucene-memory:8.3.0 -- org.apache.lucene:lucene-misc:8.3.0 -- org.apache.lucene:lucene-queries:8.3.0 -- org.apache.lucene:lucene-queryparser:8.3.0 -- org.apache.lucene:lucene-sandbox:8.3.0 -- org.apache.lucene:lucene-spatial:8.3.0 -- org.apache.lucene:lucene-spatial-extras:8.3.0 -- org.apache.lucene:lucene-spatial3d:8.3.0 -- org.apache.lucene:lucene-suggest:8.3.0 -- org.elasticsearch:elasticsearch:7.5.1 -- org.elasticsearch:elasticsearch-cli:7.5.1 -- org.elasticsearch:elasticsearch-core:7.5.1 -- org.elasticsearch:elasticsearch-geo:7.5.1 -- org.elasticsearch:elasticsearch-secure-sm:7.5.1 -- org.elasticsearch:elasticsearch-x-content:7.5.1 -- org.elasticsearch.client:elasticsearch-rest-client:7.5.1 -- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.5.1 -- org.elasticsearch.plugin:aggs-matrix-stats-client:7.5.1 -- org.elasticsearch.plugin:lang-mustache-client:7.5.1 -- org.elasticsearch.plugin:mapper-extras-client:7.5.1 -- org.elasticsearch.plugin:parent-join-client:7.5.1 -- org.elasticsearch.plugin:rank-eval-client:7.5.1 +- org.apache.lucene:lucene-analyzers-common:8.9.0 +- org.apache.lucene:lucene-backward-codecs:8.9.0 +- org.apache.lucene:lucene-core:8.9.0 +- org.apache.lucene:lucene-grouping:8.9.0 +- org.apache.lucene:lucene-highlighter:8.9.0 +- org.apache.lucene:lucene-join:8.9.0 +- org.apache.lucene:lucene-memory:8.9.0 +- org.apache.lucene:lucene-misc:8.9.0 +- org.apache.lucene:lucene-queries:8.9.0 +- org.apache.lucene:lucene-queryparser:8.9.0 +- org.apache.lucene:lucene-sandbox:8.9.0 +- org.apache.lucene:lucene-spatial:8.9.0 +- org.apache.lucene:lucene-spatial-extras:8.9.0 +- org.apache.lucene:lucene-spatial3d:8.9.0 +- org.apache.lucene:lucene-suggest:8.9.0 +- org.elasticsearch:elasticsearch:7.15.2 +- org.elasticsearch:elasticsearch-cli:7.15.2 +- org.elasticsearch:elasticsearch-core:7.15.2 +- org.elasticsearch:elasticsearch-geo:7.15.2 +- org.elasticsearch:elasticsearch-secure-sm:7.15.2 +- org.elasticsearch:elasticsearch-x-content:7.15.2 +- org.elasticsearch:elasticsearch-plugin-classloader:7.15.2 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.15.2 +- org.elasticsearch.client:elasticsearch-rest-client:7.15.2 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.15.2 +- org.elasticsearch.plugin:lang-mustache-client:7.15.2 +- org.elasticsearch.plugin:mapper-extras-client:7.15.2 +- org.elasticsearch.plugin:parent-join-client:7.15.2 +- org.elasticsearch.plugin:rank-eval-client:7.15.2 +- org.lz4:lz4-java:1.8.0 From 5420a274daf274fe65bbe4c24eab099401252971 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Fri, 3 Dec 2021 16:07:14 +0100 Subject: [PATCH 173/207] [FLINK-25189][connectors/elasticsearch] Bump Elasticsearch 6 to 6.8.20 --- .../flink-connector-elasticsearch6/pom.xml | 5 +- .../sink/Elasticsearch6SinkITCase.java | 3 +- .../pom.xml | 4 ++ .../src/main/resources/META-INF/NOTICE | 52 ++++++++++--------- 4 files changed, 35 insertions(+), 29 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index dd147d9f..64d18433 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -37,7 +37,7 @@ under the License. - 6.3.1 + 6.8.20 @@ -69,11 +69,12 @@ under the License. flink-connector-elasticsearch-base ${project.version} - + org.elasticsearch elasticsearch + org.elasticsearch.client elasticsearch-rest-high-level-client diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java index 6aa743f5..7357707e 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java @@ -33,8 +33,7 @@ class Elasticsearch6SinkITCase extends ElasticsearchSinkBaseITCase { @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer( - DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_COMMERCIAL_6)) + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)) .withPassword(ELASTICSEARCH_PASSWORD) .withLogConsumer(new Slf4jLogConsumer(LOG)); diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index c747e0c0..6aaaf4ac 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -135,6 +135,10 @@ under the License. com.fasterxml.jackson org.apache.flink.elasticsearch6.shaded.com.fasterxml.jackson + + com.github.mustachejava + org.apache.flink.elasticsearch6.shaded.com.github.mustachejava + diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index 52ea8404..de1f4f37 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -18,28 +18,30 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpclient:4.5.13 - org.apache.httpcomponents:httpcore:4.4.14 - org.apache.httpcomponents:httpcore-nio:4.4.5 -- org.apache.lucene:lucene-analyzers-common:7.3.1 -- org.apache.lucene:lucene-backward-codecs:7.3.1 -- org.apache.lucene:lucene-core:7.3.1 -- org.apache.lucene:lucene-grouping:7.3.1 -- org.apache.lucene:lucene-highlighter:7.3.1 -- org.apache.lucene:lucene-join:7.3.1 -- org.apache.lucene:lucene-memory:7.3.1 -- org.apache.lucene:lucene-misc:7.3.1 -- org.apache.lucene:lucene-queries:7.3.1 -- org.apache.lucene:lucene-queryparser:7.3.1 -- org.apache.lucene:lucene-sandbox:7.3.1 -- org.apache.lucene:lucene-spatial:7.3.1 -- org.apache.lucene:lucene-spatial-extras:7.3.1 -- org.apache.lucene:lucene-spatial3d:7.3.1 -- org.apache.lucene:lucene-suggest:7.3.1 -- org.elasticsearch:elasticsearch:6.3.1 -- org.elasticsearch:elasticsearch-cli:6.3.1 -- org.elasticsearch:elasticsearch-core:6.3.1 -- org.elasticsearch:elasticsearch-secure-sm:6.3.1 -- org.elasticsearch:elasticsearch-x-content:6.3.1 -- org.elasticsearch.client:elasticsearch-rest-client:6.3.1 -- org.elasticsearch.client:elasticsearch-rest-high-level-client:6.3.1 -- org.elasticsearch.plugin:aggs-matrix-stats-client:6.3.1 -- org.elasticsearch.plugin:parent-join-client:6.3.1 -- org.elasticsearch.plugin:rank-eval-client:6.3.1 +- org.apache.lucene:lucene-analyzers-common:7.7.3 +- org.apache.lucene:lucene-backward-codecs:7.7.3 +- org.apache.lucene:lucene-core:7.7.3 +- org.apache.lucene:lucene-grouping:7.7.3 +- org.apache.lucene:lucene-highlighter:7.7.3 +- org.apache.lucene:lucene-join:7.7.3 +- org.apache.lucene:lucene-memory:7.7.3 +- org.apache.lucene:lucene-misc:7.7.3 +- org.apache.lucene:lucene-queries:7.7.3 +- org.apache.lucene:lucene-queryparser:7.7.3 +- org.apache.lucene:lucene-sandbox:7.7.3 +- org.apache.lucene:lucene-spatial:7.7.3 +- org.apache.lucene:lucene-spatial-extras:7.7.3 +- org.apache.lucene:lucene-spatial3d:7.7.3 +- org.apache.lucene:lucene-suggest:7.7.3 +- org.elasticsearch:elasticsearch:6.8.20 +- org.elasticsearch:elasticsearch-cli:6.8.20 +- org.elasticsearch:elasticsearch-core:6.8.20 +- org.elasticsearch:elasticsearch-secure-sm:6.8.20 +- org.elasticsearch:elasticsearch-x-content:6.8.20 +- org.elasticsearch.client:elasticsearch-rest-client:6.8.20 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:6.8.20 +- org.elasticsearch.plugin:aggs-matrix-stats-client:6.8.20 +- org.elasticsearch.plugin:parent-join-client:6.8.20 +- org.elasticsearch.plugin:rank-eval-client:6.8.20 +- org.elasticsearch.plugin:lang-mustache-client:6.8.20 +- com.github.spullara.mustache.java:compiler:0.9.3 From c72ed326fd02cf4ed9474a63784610a2577deccf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 9 Dec 2021 11:20:04 +0100 Subject: [PATCH 174/207] [FLINK-25223][connectors/elasticsearch] Disable Elasticsearch 7 Testcontainer tests due to frequent CI failures This closes #18062. --- .../connector/elasticsearch/sink/ElasticsearchWriterITCase.java | 2 ++ .../connector/elasticsearch/sink/Elasticsearch7SinkITCase.java | 2 ++ .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 2 ++ .../connectors/elasticsearch7/ElasticsearchSinkITCase.java | 2 ++ 4 files changed, 8 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 2fa75023..241ef6a7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -47,6 +47,7 @@ import org.elasticsearch.core.TimeValue; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; @@ -69,6 +70,7 @@ /** Tests for {@link ElasticsearchWriter}. */ @Testcontainers +@Disabled @ExtendWith(TestLoggerExtension.class) class ElasticsearchWriterITCase { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java index fa4638c2..b7e2a91b 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.client.RestHighLevelClient; +import org.junit.jupiter.api.Disabled; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; @@ -29,6 +30,7 @@ /** Tests for {@link ElasticsearchSink}. */ @Testcontainers +@Disabled class Elasticsearch7SinkITCase extends ElasticsearchSinkBaseITCase { @Container diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 8c6bbb01..543cb6ea 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -25,6 +25,7 @@ import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; +import org.junit.jupiter.api.Disabled; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -36,6 +37,7 @@ /** IT tests for {@link ElasticsearchDynamicSink}. */ @Testcontainers +@Disabled public class Elasticsearch7DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { @Container diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 65332a86..264bd331 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.utility.DockerImageName; @@ -39,6 +40,7 @@ import java.util.List; /** IT cases for the {@link ElasticsearchSink}. */ +@Ignore public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { From eac7115b21deb62684d669db428ae5632eba1bcb Mon Sep 17 00:00:00 2001 From: jinfeng Date: Mon, 13 Dec 2021 19:52:37 +0800 Subject: [PATCH 175/207] [FLINK-21068][connectors/elasticsearch] Support 'connection.request-timeout','connection.timeout', 'socket.timeout' options for elasticsearch connector. --- .../sink/ElasticsearchSinkBuilderBase.java | 49 ++++++++++++++++++- .../sink/ElasticsearchWriter.java | 20 ++++++++ .../sink/NetworkClientConfig.java | 26 +++++++++- .../table/ElasticsearchConfiguration.java | 15 ++++++ .../table/ElasticsearchConnectorOptions.java | 21 ++++++++ .../table/ElasticsearchDynamicSink.java | 13 +++++ .../ElasticsearchDynamicSinkFactoryBase.java | 6 +++ .../ElasticsearchSinkBuilderBaseTest.java | 13 +++++ .../sink/ElasticsearchWriterITCase.java | 2 +- 9 files changed, 162 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index f187cb6d..fe64c949 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -54,6 +54,9 @@ public abstract class ElasticsearchSinkBuilderBase< private String username; private String password; private String connectionPathPrefix; + private Integer connectionTimeout; + private Integer connectionRequestTimeout; + private Integer socketTimeout; protected ElasticsearchSinkBuilderBase() {} @@ -217,6 +220,44 @@ public B setConnectionPathPrefix(String prefix) { return self(); } + /** + * Sets the timeout for requesting the connection of the Elasticsearch cluster from the + * connection manager. + * + * @param timeout for the connection request + * @return this builder + */ + public B setConnectionRequestTimeout(int timeout) { + checkState(timeout >= 0, "Connection request timeout must be larger than or equal to 0."); + this.connectionRequestTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for establishing a connection of the Elasticsearch cluster. + * + * @param timeout for the connection + * @return this builder + */ + public B setConnectionTimeout(int timeout) { + checkState(timeout >= 0, "Connection timeout must be larger than or equal to 0."); + this.connectionTimeout = timeout; + return self(); + } + + /** + * Sets the timeout for waiting for data or, put differently, a maximum period inactivity + * between two consecutive data packets. + * + * @param timeout for the socket + * @return this builder + */ + public B setSocketTimeout(int timeout) { + checkState(timeout >= 0, "Socket timeout must be larger than or equal to 0."); + this.socketTimeout = timeout; + return self(); + } + protected abstract BulkProcessorBuilderFactory getBulkProcessorBuilderFactory(); /** @@ -247,7 +288,13 @@ public ElasticsearchSink build() { private NetworkClientConfig buildNetworkClientConfig() { checkArgument(!hosts.isEmpty(), "Hosts cannot be empty."); - return new NetworkClientConfig(username, password, connectionPathPrefix); + return new NetworkClientConfig( + username, + password, + connectionPathPrefix, + connectionRequestTimeout, + connectionTimeout, + socketTimeout); } private BulkProcessorConfig buildBulkProcessorConfig() { diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index f984312b..53723bc7 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -169,6 +169,26 @@ private static RestClientBuilder configureRestClientBuilder( httpClientBuilder -> httpClientBuilder.setDefaultCredentialsProvider(credentialsProvider)); } + if (networkClientConfig.getConnectionRequestTimeout() != null + || networkClientConfig.getConnectionTimeout() != null + || networkClientConfig.getSocketTimeout() != null) { + builder.setRequestConfigCallback( + requestConfigBuilder -> { + if (networkClientConfig.getConnectionRequestTimeout() != null) { + requestConfigBuilder.setConnectionRequestTimeout( + networkClientConfig.getConnectionRequestTimeout()); + } + if (networkClientConfig.getConnectionTimeout() != null) { + requestConfigBuilder.setConnectTimeout( + networkClientConfig.getConnectionTimeout()); + } + if (networkClientConfig.getSocketTimeout() != null) { + requestConfigBuilder.setSocketTimeout( + networkClientConfig.getSocketTimeout()); + } + return requestConfigBuilder; + }); + } return builder; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java index e093008a..5ae05108 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java @@ -27,14 +27,23 @@ class NetworkClientConfig implements Serializable { @Nullable private final String username; @Nullable private final String password; @Nullable private final String connectionPathPrefix; + @Nullable private final Integer connectionRequestTimeout; + @Nullable private final Integer connectionTimeout; + @Nullable private final Integer socketTimeout; NetworkClientConfig( @Nullable String username, @Nullable String password, - @Nullable String connectionPathPrefix) { + @Nullable String connectionPathPrefix, + @Nullable Integer connectionRequestTimeout, + @Nullable Integer connectionTimeout, + @Nullable Integer socketTimeout) { this.username = username; this.password = password; this.connectionPathPrefix = connectionPathPrefix; + this.connectionRequestTimeout = connectionRequestTimeout; + this.connectionTimeout = connectionTimeout; + this.socketTimeout = socketTimeout; } @Nullable @@ -47,6 +56,21 @@ public String getPassword() { return password; } + @Nullable + public Integer getConnectionRequestTimeout() { + return connectionRequestTimeout; + } + + @Nullable + public Integer getConnectionTimeout() { + return connectionTimeout; + } + + @Nullable + public Integer getSocketTimeout() { + return socketTimeout; + } + @Nullable public String getConnectionPathPrefix() { return connectionPathPrefix; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java index ee6488e4..e684b01f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java @@ -39,11 +39,14 @@ import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -104,6 +107,18 @@ public Optional getPathPrefix() { return config.getOptional(CONNECTION_PATH_PREFIX_OPTION); } + public Optional getConnectionRequestTimeout() { + return config.getOptional(CONNECTION_REQUEST_TIMEOUT); + } + + public Optional getConnectionTimeout() { + return config.getOptional(CONNECTION_TIMEOUT); + } + + public Optional getSocketTimeout() { + return config.getOptional(SOCKET_TIMEOUT); + } + public List getHosts() { return config.get(HOSTS_OPTION).stream() .map(ElasticsearchConfiguration::validateAndParseHostsString) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java index 5cff4ee7..672f0727 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -111,6 +111,27 @@ public class ElasticsearchConnectorOptions { .noDefaultValue() .withDescription("Prefix string to be added to every REST communication."); + public static final ConfigOption CONNECTION_REQUEST_TIMEOUT = + ConfigOptions.key("connection.request-timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The timeout for requesting a connection from the connection manager."); + + public static final ConfigOption CONNECTION_TIMEOUT = + ConfigOptions.key("connection.timeout") + .durationType() + .noDefaultValue() + .withDescription("The timeout for establishing a connection."); + + public static final ConfigOption SOCKET_TIMEOUT = + ConfigOptions.key("socket.timeout") + .durationType() + .noDefaultValue() + .withDescription( + "The socket timeout (SO_TIMEOUT) for waiting for data or, put differently," + + "a maximum period inactivity between two consecutive data packets."); + public static final ConfigOption FORMAT_OPTION = ConfigOptions.key("format") .stringType() diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index e5f94f19..0938e024 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -144,6 +144,19 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { builder.setConnectionPathPrefix(config.getPathPrefix().get()); } + if (config.getConnectionRequestTimeout().isPresent()) { + builder.setConnectionRequestTimeout( + (int) config.getConnectionRequestTimeout().get().getSeconds()); + } + + if (config.getConnectionTimeout().isPresent()) { + builder.setConnectionTimeout((int) config.getConnectionTimeout().get().getSeconds()); + } + + if (config.getSocketTimeout().isPresent()) { + builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds()); + } + return SinkProvider.of(builder.build()); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index e124ad6b..8028284e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -54,12 +54,15 @@ import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_SIZE_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_REQUEST_TIMEOUT; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.DELIVERY_GUARANTEE_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.elasticsearch.common.Strings.capitalize; @@ -209,6 +212,9 @@ public Set> optionalOptions() { BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, CONNECTION_PATH_PREFIX_OPTION, + CONNECTION_REQUEST_TIMEOUT, + CONNECTION_TIMEOUT, + SOCKET_TIMEOUT, FORMAT_OPTION, DELIVERY_GUARANTEE_OPTION, PASSWORD_OPTION, diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index e65d844a..83f78714 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -77,6 +77,19 @@ void testThrowIfEmitterNotSet() { () -> createEmptyBuilder().setHosts(new HttpHost("localhost:3000")).build()); } + @Test + void testThrowIfSetInvalidTimeouts() { + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionRequestTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setConnectionTimeout(-1).build()); + assertThrows( + IllegalStateException.class, + () -> createEmptyBuilder().setSocketTimeout(-1).build()); + } + abstract B createEmptyBuilder(); abstract B createMinimalBuilder(); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 241ef6a7..222f8b38 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -248,7 +248,7 @@ private ElasticsearchWriter> createWriter( flushOnCheckpoint, bulkProcessorConfig, new TestBulkProcessorBuilderFactory(), - new NetworkClientConfig(null, null, null), + new NetworkClientConfig(null, null, null, null, null, null), metricGroup, new TestMailbox()); } From 81d2209f4c87c8dfa226f260eff97330e999ea83 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 9 Dec 2021 14:13:02 +0100 Subject: [PATCH 176/207] [FLINK-25223][connectors/elasticsearch] Limit Elasticsearch testcontainer memory allocation --- .../elasticsearch/ElasticsearchUtil.java | 64 +++++++++++++++++++ .../sink/ElasticsearchWriterITCase.java | 9 +-- .../sink/Elasticsearch6SinkITCase.java | 8 +-- .../Elasticsearch6DynamicSinkITCase.java | 9 ++- .../ElasticsearchSinkITCase.java | 9 ++- .../sink/Elasticsearch7SinkITCase.java | 10 +-- .../Elasticsearch7DynamicSinkITCase.java | 11 +++- .../ElasticsearchSinkITCase.java | 11 ++-- 8 files changed, 103 insertions(+), 28 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java new file mode 100644 index 00000000..e3836801 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java @@ -0,0 +1,64 @@ +/* + * 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.connector.elasticsearch; + +import org.apache.flink.annotation.Internal; + +import org.slf4j.Logger; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +/** Collection of utility methods for Elasticsearch tests. */ +@Internal +public class ElasticsearchUtil { + + private ElasticsearchUtil() {} + + /** + * Creates a preconfigured {@link ElasticsearchContainer} with limited memory allocation and + * aligns the internal Elasticsearch log levels with the ones used by the capturing logger. + * + * @param dockerImageVersion describing the Elasticsearch image + * @param log to derive the log level from + * @return configured Elasticsearch container + */ + public static ElasticsearchContainer createElasticsearchContainer( + String dockerImageVersion, Logger log) { + String logLevel; + if (log.isTraceEnabled()) { + logLevel = "TRACE"; + } else if (log.isDebugEnabled()) { + logLevel = "DEBUG"; + } else if (log.isInfoEnabled()) { + logLevel = "INFO"; + } else if (log.isWarnEnabled()) { + logLevel = "WARN"; + } else if (log.isErrorEnabled()) { + logLevel = "ERROR"; + } else { + logLevel = "OFF"; + } + + return new ElasticsearchContainer(DockerImageName.parse(dockerImageVersion)) + .withEnv("ES_JAVA_OPTS", "-Xms2g -Xmx2g") + .withEnv("logger.org.elasticsearch", logLevel) + .withLogConsumer(new Slf4jLogConsumer(log)); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 222f8b38..0f15bef0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -19,6 +19,7 @@ import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; @@ -47,16 +48,13 @@ import org.elasticsearch.core.TimeValue; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; import java.io.IOException; import java.util.Collections; @@ -70,7 +68,6 @@ /** Tests for {@link ElasticsearchWriter}. */ @Testcontainers -@Disabled @ExtendWith(TestLoggerExtension.class) class ElasticsearchWriterITCase { @@ -78,8 +75,8 @@ class ElasticsearchWriterITCase { @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)) - .withLogConsumer(new Slf4jLogConsumer(LOG)); + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); private RestHighLevelClient client; private TestClientBase context; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java index 7357707e..c947cda9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java @@ -18,14 +18,13 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.client.RestHighLevelClient; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; /** Tests for {@link ElasticsearchSink}. */ @Testcontainers @@ -33,9 +32,8 @@ class Elasticsearch6SinkITCase extends ElasticsearchSinkBaseITCase { @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)) - .withPassword(ELASTICSEARCH_PASSWORD) - .withLogConsumer(new Slf4jLogConsumer(LOG)); + ElasticsearchUtil.createElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_6, LOG) + .withPassword(ELASTICSEARCH_PASSWORD); @Override String getElasticsearchHttpHostAddress() { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 06b45915..055a822a 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -18,12 +18,15 @@ package org.apache.flink.connector.elasticsearch.table; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -37,11 +40,15 @@ @Testcontainers public class Elasticsearch6DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { + private static final Logger LOG = + LoggerFactory.getLogger(Elasticsearch6DynamicSinkITCase.class); + private static final String DOCUMENT_TYPE = "MyType"; @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_6); + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_6, LOG); @Override String getElasticsearchHttpHostAddress() { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 0637ca52..b29e9585 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.elasticsearch6; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; @@ -32,8 +33,9 @@ import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; import java.util.ArrayList; import java.util.List; @@ -42,9 +44,12 @@ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_6, LOG); @Override protected String getClusterName() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java index b7e2a91b..62ed8dea 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java @@ -18,26 +18,22 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.client.RestHighLevelClient; -import org.junit.jupiter.api.Disabled; -import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; -import org.testcontainers.utility.DockerImageName; /** Tests for {@link ElasticsearchSink}. */ @Testcontainers -@Disabled class Elasticsearch7SinkITCase extends ElasticsearchSinkBaseITCase { @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)) - .withPassword(ELASTICSEARCH_PASSWORD) - .withLogConsumer(new Slf4jLogConsumer(LOG)); + ElasticsearchUtil.createElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_7, LOG) + .withPassword(ELASTICSEARCH_PASSWORD); @Override String getElasticsearchHttpHostAddress() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index 543cb6ea..caddcda9 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.elasticsearch.table; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.util.DockerImageVersions; import org.elasticsearch.action.get.GetRequest; @@ -25,7 +26,8 @@ import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.junit.jupiter.api.Disabled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.junit.jupiter.Container; import org.testcontainers.junit.jupiter.Testcontainers; @@ -37,12 +39,15 @@ /** IT tests for {@link ElasticsearchDynamicSink}. */ @Testcontainers -@Disabled public class Elasticsearch7DynamicSinkITCase extends ElasticsearchDynamicSinkBaseITCase { + private static final Logger LOG = + LoggerFactory.getLogger(Elasticsearch7DynamicSinkITCase.class); + @Container private static final ElasticsearchContainer ES_CONTAINER = - new ElasticsearchContainer(DockerImageVersions.ELASTICSEARCH_7); + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); @Override String getElasticsearchHttpHostAddress() { diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index 264bd331..e5f89bd0 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.connectors.elasticsearch7; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; @@ -31,22 +32,24 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; -import org.junit.Ignore; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testcontainers.elasticsearch.ElasticsearchContainer; -import org.testcontainers.utility.DockerImageName; import java.util.ArrayList; import java.util.List; /** IT cases for the {@link ElasticsearchSink}. */ -@Ignore public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkITCase.class); + @ClassRule public static ElasticsearchContainer elasticsearchContainer = - new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + ElasticsearchUtil.createElasticsearchContainer( + DockerImageVersions.ELASTICSEARCH_7, LOG); @Override protected String getClusterName() { From 8a5c1174d31169fd04fa393487b936c6cf1add39 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Wed, 8 Dec 2021 17:13:20 +0800 Subject: [PATCH 177/207] [FLINK-25105][checkpoint] Enables final checkpoint by default This closes #18068. --- .../elasticsearch/sink/ElasticsearchSinkBaseITCase.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java index 757bf311..927f672d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java @@ -21,11 +21,9 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -172,9 +170,7 @@ private void runTest( .setDeliveryGuarantee(deliveryGuarantee) .build(); - final Configuration config = new Configuration(); - config.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); - final StreamExecutionEnvironment env = new LocalStreamEnvironment(config); + final StreamExecutionEnvironment env = new LocalStreamEnvironment(); env.enableCheckpointing(100L); if (!allowRestarts) { env.setRestartStrategy(RestartStrategies.noRestart()); From 440f632046ce48822007ee106e84e3619621232f Mon Sep 17 00:00:00 2001 From: Ada Wong Date: Tue, 7 Dec 2021 10:41:50 +0800 Subject: [PATCH 178/207] [FLINK-25141][connector/elasticsearch] Add sink parallelism option --- .../table/ElasticsearchConfiguration.java | 5 ++++ .../table/ElasticsearchDynamicSink.java | 2 +- .../ElasticsearchDynamicSinkFactoryBase.java | 4 ++- .../elasticsearch/ElasticsearchUtil.java | 28 ++++++++++++++++++ .../ElasticsearchDynamicSinkBaseITCase.java | 29 ++----------------- ...asticsearchDynamicSinkFactoryBaseTest.java | 21 ++++++++++++++ 6 files changed, 60 insertions(+), 29 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java index e684b01f..3bedea14 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java @@ -48,6 +48,7 @@ import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; import static org.apache.flink.util.Preconditions.checkNotNull; /** Elasticsearch base configuration. */ @@ -125,6 +126,10 @@ public List getHosts() { .collect(Collectors.toList()); } + public Optional getParallelism() { + return config.getOptional(SINK_PARALLELISM); + } + private static HttpHost validateAndParseHostsString(String host) { try { HttpHost httpHost = HttpHost.create(host); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index 0938e024..fbab8b78 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -157,7 +157,7 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds()); } - return SinkProvider.of(builder.build()); + return SinkProvider.of(builder.build(), config.getParallelism().orElse(null)); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index 8028284e..677a8e3c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -64,6 +64,7 @@ import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.SOCKET_TIMEOUT; import static org.apache.flink.connector.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; import static org.apache.flink.util.Preconditions.checkNotNull; import static org.elasticsearch.common.Strings.capitalize; @@ -218,7 +219,8 @@ public Set> optionalOptions() { FORMAT_OPTION, DELIVERY_GUARANTEE_OPTION, PASSWORD_OPTION, - USERNAME_OPTION) + USERNAME_OPTION, + SINK_PARALLELISM) .collect(Collectors.toSet()); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java index e3836801..7b856c2d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java @@ -19,6 +19,10 @@ package org.apache.flink.connector.elasticsearch; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.slf4j.Logger; import org.testcontainers.containers.output.Slf4jLogConsumer; @@ -61,4 +65,28 @@ public static ElasticsearchContainer createElasticsearchContainer( .withEnv("logger.org.elasticsearch", logLevel) .withLogConsumer(new Slf4jLogConsumer(log)); } + + /** A mock {@link DynamicTableSink.Context} for Elasticsearch tests. */ + public static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index e256a51a..6392ffcc 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -19,8 +19,8 @@ package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; @@ -35,8 +35,6 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.types.RowKind; import org.apache.http.HttpHost; @@ -117,7 +115,7 @@ public void testWritingDocuments() throws Exception { sinkFactory .createDynamicTableSink( getPrefilledTestContext(index).withSchema(schema).build()) - .getSinkRuntimeProvider(new MockContext()); + .getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; final Sink sink = sinkProvider.createSink(); @@ -301,27 +299,4 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { expectedMap.put("b", "2012-12-12 12:12:12"); Assertions.assertEquals(response, expectedMap); } - - private static class MockContext implements DynamicTableSink.Context { - @Override - public boolean isBounded() { - return false; - } - - @Override - public TypeInformation createTypeInformation(DataType consumedDataType) { - return null; - } - - @Override - public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { - return null; - } - - @Override - public DynamicTableSink.DataStructureConverter createDataStructureConverter( - DataType consumedDataType) { - return null; - } - } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java index 29e3d212..3739bb40 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -19,12 +19,15 @@ package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkProvider; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -34,6 +37,9 @@ import java.util.Arrays; import java.util.Collections; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.assertj.core.api.Assertions.assertThat; + /** Tests for validation in {@link ElasticsearchDynamicSinkFactoryBase}. */ @ExtendWith(TestLoggerExtension.class) abstract class ElasticsearchDynamicSinkFactoryBaseTest { @@ -227,4 +233,19 @@ public void validateWrongCredential() { "") .build())); } + + @Test + public void testSinkParallelism() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + DynamicTableSink sink = + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption(SINK_PARALLELISM.key(), "2") + .build()); + assertThat(sink).isInstanceOf(ElasticsearchDynamicSink.class); + ElasticsearchDynamicSink esSink = (ElasticsearchDynamicSink) sink; + SinkProvider provider = + (SinkProvider) esSink.getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); + assertThat(2).isEqualTo(provider.getParallelism().get()); + } } From 93cbcb6283698cf62e5dfb5e2c9d74519e197028 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 6 Jan 2022 10:24:42 +0100 Subject: [PATCH 179/207] [FLINK-25390][table-common] Introduce forwardOptions for table and format factories --- .../apache/flink/connector/elasticsearch/table/TestContext.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java index edea4fce..0d7bc551 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java @@ -59,6 +59,7 @@ DynamicTableFactory.Context build() { Collections.emptyList(), options), schema), + Collections.emptyMap(), new Configuration(), TestContext.class.getClassLoader(), false); From 014d9216140ca931c4da986ff84234e81cee2cd6 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Tue, 18 Jan 2022 21:21:17 +0800 Subject: [PATCH 180/207] [FLINK-25712][connectors/testing-framework] Merge flink-connector-testing module to flink-connector-test-utils module This closes #18437. --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- .../elasticsearch/sink/ElasticsearchSinkBaseITCase.java | 2 +- .../elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java | 2 +- .../connector/elasticsearch/sink/ElasticsearchWriterITCase.java | 2 +- .../elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java | 2 +- .../table/ElasticsearchDynamicSinkFactoryBaseTest.java | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index e5c30d9a..6cc24be8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -119,7 +119,7 @@ under the License. org.apache.flink - flink-connector-testing + flink-connector-test-utils ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java index 927f672d..af87158b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java @@ -22,10 +22,10 @@ import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.LocalStreamEnvironment; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index 83f78714..695f85c1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -18,7 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; +import org.apache.flink.util.TestLoggerExtension; import org.apache.http.HttpHost; import org.junit.jupiter.api.DynamicTest; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 0f15bef0..4fea99ee 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; -import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; @@ -30,6 +29,7 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.ThrowingRunnable; import org.apache.http.HttpHost; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index 6392ffcc..a6ab6bea 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.connector.sink.Sink; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; -import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; @@ -36,6 +35,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.util.TestLoggerExtension; import org.apache.http.HttpHost; import org.elasticsearch.client.RestClient; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java index 3739bb40..c151e719 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; -import org.apache.flink.connectors.test.common.junit.extensions.TestLoggerExtension; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; @@ -28,6 +27,7 @@ import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 64d18433..7d5161e9 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -134,7 +134,7 @@ under the License. org.apache.flink - flink-connector-testing + flink-connector-test-utils ${project.version} test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 138c5d29..e8e45f01 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -137,7 +137,7 @@ under the License. org.apache.flink - flink-connector-testing + flink-connector-test-utils ${project.version} test From 0e34e32cba9574ede410232c02f174f5cce450f3 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Thu, 6 Jan 2022 14:52:38 +0100 Subject: [PATCH 181/207] [FLINK-25391][connector-elasticsearch] Forward catalog table options --- .../ElasticsearchDynamicSinkFactoryBase.java | 50 ++++++++++++------- .../Elasticsearch6DynamicSinkFactory.java | 21 +++++--- 2 files changed, 44 insertions(+), 27 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index 677a8e3c..af358883 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -29,7 +28,6 @@ import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.factories.DynamicTableFactory; import org.apache.flink.table.factories.DynamicTableSinkFactory; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.SerializationFormatFactory; @@ -83,7 +81,7 @@ public ElasticsearchDynamicSinkFactoryBase( } @Nullable - String getDocumentType(Context context) { + String getDocumentType(ElasticsearchConfiguration configuration) { return null; // document type is only set in Elasticsearch versions < 7 } @@ -91,10 +89,14 @@ String getDocumentType(Context context) { public DynamicTableSink createDynamicTableSink(Context context) { List primaryKeyLogicalTypesWithIndex = getPrimaryKeyLogicalTypesWithIndex(context); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); EncodingFormat> format = - getValidatedEncodingFormat(this, context); + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); - ElasticsearchConfiguration config = getConfiguration(context); + ElasticsearchConfiguration config = getConfiguration(helper); + helper.validate(); validateConfiguration(config); return new ElasticsearchDynamicSink( @@ -104,12 +106,11 @@ public DynamicTableSink createDynamicTableSink(Context context) { context.getPhysicalRowDataType(), capitalize(factoryIdentifier), sinkBuilderSupplier, - getDocumentType(context)); + getDocumentType(config)); } - ElasticsearchConfiguration getConfiguration(Context context) { - return new ElasticsearchConfiguration( - Configuration.fromMap(context.getCatalogTable().getOptions())); + ElasticsearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) { + return new ElasticsearchConfiguration(helper.getOptions()); } void validateConfiguration(ElasticsearchConfiguration config) { @@ -161,16 +162,6 @@ static void validate(boolean condition, Supplier message) { } } - EncodingFormat> getValidatedEncodingFormat( - DynamicTableFactory factory, DynamicTableFactory.Context context) { - final FactoryUtil.TableFactoryHelper helper = - FactoryUtil.createTableFactoryHelper(factory, context); - final EncodingFormat> format = - helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); - helper.validate(); - return format; - } - List getPrimaryKeyLogicalTypesWithIndex(Context context) { DataType physicalRowDataType = context.getPhysicalRowDataType(); int[] primaryKeyIndexes = context.getPrimaryKeyIndexes(); @@ -224,6 +215,27 @@ public Set> optionalOptions() { .collect(Collectors.toSet()); } + @Override + public Set> forwardOptions() { + return Stream.of( + HOSTS_OPTION, + INDEX_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION, + KEY_DELIMITER_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_MAX_SIZE_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_PATH_PREFIX_OPTION, + CONNECTION_REQUEST_TIMEOUT, + CONNECTION_TIMEOUT, + SOCKET_TIMEOUT) + .collect(Collectors.toSet()); + } + @Override public String factoryIdentifier() { return factoryIdentifier; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index 2bb2c8a5..69576978 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -20,14 +20,16 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder; import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.util.StringUtils; import javax.annotation.Nullable; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.connector.elasticsearch.table.Elasticsearch6ConnectorOptions.DOCUMENT_TYPE_OPTION; @@ -41,17 +43,14 @@ public Elasticsearch6DynamicSinkFactory() { } @Override - ElasticsearchConfiguration getConfiguration(Context context) { - return new Elasticsearch6Configuration( - Configuration.fromMap(context.getCatalogTable().getOptions())); + ElasticsearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) { + return new Elasticsearch6Configuration(helper.getOptions()); } @Nullable @Override - String getDocumentType(Context context) { - Elasticsearch6Configuration config = - (Elasticsearch6Configuration) getConfiguration(context); - return config.getDocumentType(); + String getDocumentType(ElasticsearchConfiguration configuration) { + return ((Elasticsearch6Configuration) configuration).getDocumentType(); } @Override @@ -69,4 +68,10 @@ public Set> requiredOptions() { requiredOptions.add(DOCUMENT_TYPE_OPTION); return requiredOptions; } + + @Override + public Set> forwardOptions() { + return Stream.concat(super.forwardOptions().stream(), Stream.of(DOCUMENT_TYPE_OPTION)) + .collect(Collectors.toSet()); + } } From bb3a894e09c3f840988b423d939300042895192b Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Wed, 26 Jan 2022 13:10:33 +0100 Subject: [PATCH 182/207] [FLINK-25754][elsaticsearch] Remove unused private class --- .../elasticsearch/table/KeyExtractor.java | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java index 6beff6d7..f369ee00 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.api.TableColumn; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; @@ -60,24 +59,6 @@ public String apply(RowData rowData) { return builder.toString(); } - private static class ColumnWithIndex { - public TableColumn column; - public int index; - - public ColumnWithIndex(TableColumn column, int index) { - this.column = column; - this.index = index; - } - - public LogicalType getType() { - return column.getType().getLogicalType(); - } - - public int getIndex() { - return index; - } - } - public static SerializableFunction createKeyExtractor( List primaryKeyTypesWithIndex, String keyDelimiter) { if (!primaryKeyTypesWithIndex.isEmpty()) { From 00945bbd122302f946abc2bfe67130043610e911 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Thu, 3 Feb 2022 16:24:44 +0100 Subject: [PATCH 183/207] [FLINK-25892][elasticsearch6][test] add ArchUnit tests for test code --- .../2626cbc6-34aa-46c3-a57a-a4f934cc4e9e | 0 .../cace802d-2fe7-4eef-aef1-8ba8e335f854 | 3 + .../archunit-violations/stored.rules | 4 ++ .../flink-connector-elasticsearch6/pom.xml | 7 +++ .../TestCodeArchitectureTest.java | 57 +++++++++++++++++++ .../src/test/resources/archunit.properties | 31 ++++++++++ 6 files changed, 102 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch6/archunit-violations/2626cbc6-34aa-46c3-a57a-a4f934cc4e9e create mode 100644 flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 create mode 100644 flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/2626cbc6-34aa-46c3-a57a-a4f934cc4e9e b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/2626cbc6-34aa-46c3-a57a-a4f934cc4e9e new file mode 100644 index 00000000..e69de29b diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 new file mode 100644 index 00000000..bddadc6d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 @@ -0,0 +1,3 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules new file mode 100644 index 00000000..5c13f4d5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu Feb 03 14:58:57 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=cace802d-2fe7-4eef-aef1-8ba8e335f854 +classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=2626cbc6-34aa-46c3-a57a-a4f934cc4e9e diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 7d5161e9..6a5b5e86 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -179,6 +179,13 @@ under the License. test + + + + org.apache.flink + flink-architecture-tests-test + test + diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..4f99f9ee --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,57 @@ +/* + * 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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.core.importer.Location; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +import java.util.regex.Pattern; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch6" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + TestCodeArchitectureTest.IncludeES6ImportOption.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); + + /** Only include ES6 related locations. */ + public static final class IncludeES6ImportOption implements ImportOption { + private static final Pattern ES6 = Pattern.compile(".*elasticsearch6.*"); + + @Override + public boolean includes(Location location) { + return location.matches(ES6); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations From b30b7d373290dbcb986726127a9d7f068203ebf5 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Thu, 3 Feb 2022 16:25:38 +0100 Subject: [PATCH 184/207] [FLINK-25892][elasticsearch7][test] add ArchUnit tests for test code --- .../6802607d-5f0d-4f36-bb90-9566760ecb89 | 0 .../f1068e32-05ee-4824-8b9e-953d6fd42958 | 3 + .../archunit-violations/stored.rules | 4 ++ .../flink-connector-elasticsearch7/pom.xml | 7 +++ .../TestCodeArchitectureTest.java | 57 +++++++++++++++++++ .../src/test/resources/archunit.properties | 31 ++++++++++ 6 files changed, 102 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch7/archunit-violations/6802607d-5f0d-4f36-bb90-9566760ecb89 create mode 100644 flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 create mode 100644 flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/6802607d-5f0d-4f36-bb90-9566760ecb89 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/6802607d-5f0d-4f36-bb90-9566760ecb89 new file mode 100644 index 00000000..e69de29b diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 new file mode 100644 index 00000000..ab17befe --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 @@ -0,0 +1,3 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules new file mode 100644 index 00000000..f38293c8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu Feb 03 15:06:52 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f1068e32-05ee-4824-8b9e-953d6fd42958 +classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=6802607d-5f0d-4f36-bb90-9566760ecb89 diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index e8e45f01..74870223 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -176,6 +176,13 @@ under the License. test + + + + org.apache.flink + flink-architecture-tests-test + test + diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..53898b90 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,57 @@ +/* + * 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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.core.importer.Location; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +import java.util.regex.Pattern; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch7" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + TestCodeArchitectureTest.IncludeES7ImportOption.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); + + /** Only include ES7 related locations. */ + public static final class IncludeES7ImportOption implements ImportOption { + private static final Pattern ES7 = Pattern.compile(".*elasticsearch7.*"); + + @Override + public boolean includes(Location location) { + return location.matches(ES7); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations From 716be6f505ae09e31a2e771808f7fd97ebc33e4d Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Thu, 3 Feb 2022 16:26:18 +0100 Subject: [PATCH 185/207] [FLINK-25892][elasticsearch-base][test] add ArchUnit tests for test code --- .../1dc63ea4-0c1a-473b-b904-a5141706cac1 | 0 .../f527dc38-29ae-4dfd-b8f6-9ee4628fec79 | 1 + .../archunit-violations/stored.rules | 4 ++ .../pom.xml | 7 +++ .../TestCodeArchitectureTest.java | 43 +++++++++++++++++++ .../src/test/resources/archunit.properties | 31 +++++++++++++ 6 files changed, 86 insertions(+) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1dc63ea4-0c1a-473b-b904-a5141706cac1 create mode 100644 flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 create mode 100644 flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1dc63ea4-0c1a-473b-b904-a5141706cac1 b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1dc63ea4-0c1a-473b-b904-a5141706cac1 new file mode 100644 index 00000000..e69de29b diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 new file mode 100644 index 00000000..e9fa01bd --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 @@ -0,0 +1 @@ +org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules new file mode 100644 index 00000000..ffb1cfd9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu Feb 03 15:07:30 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f527dc38-29ae-4dfd-b8f6-9ee4628fec79 +classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=1dc63ea4-0c1a-473b-b904-a5141706cac1 diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index 6cc24be8..b42f97c0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -174,6 +174,13 @@ under the License. test + + + + org.apache.flink + flink-architecture-tests-test + test + diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java new file mode 100644 index 00000000..64b8573d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -0,0 +1,43 @@ +/* + * 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.architecture; + +import org.apache.flink.architecture.common.ImportOptions; + +import com.tngtech.archunit.core.importer.ImportOption; +import com.tngtech.archunit.junit.AnalyzeClasses; +import com.tngtech.archunit.junit.ArchTest; +import com.tngtech.archunit.junit.ArchTests; + +/** Architecture tests for test code. */ +@AnalyzeClasses( + packages = { + "org.apache.flink.connector.elasticsearch", + "org.apache.flink.streaming.connectors.elasticsearch" + }, + importOptions = { + ImportOption.OnlyIncludeTests.class, + ImportOptions.ExcludeScalaImportOption.class, + ImportOptions.ExcludeShadedImportOption.class + }) +public class TestCodeArchitectureTest { + + @ArchTest + public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties new file mode 100644 index 00000000..15be88c9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# By default we allow removing existing violations, but fail when new violations are added. +freeze.store.default.allowStoreUpdate=true + +# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. +#freeze.store.default.allowStoreCreation=true + +# Enable this to add allow new violations to be recorded. +# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new +# violation, please try to avoid creating the violation. If the violation was created due to a +# shortcoming of the rule, file a JIRA issue so the rule can be improved. +#freeze.refreeze=true + +freeze.store.default.path=archunit-violations From 8adace43b38d00e3174ba9a8840a72446893561f Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Tue, 8 Feb 2022 16:25:33 +0100 Subject: [PATCH 186/207] [FLINK-25571] Update Elasticsearch Sink to use decomposed interfaces --- .../sink/ElasticsearchEmitter.java | 2 +- .../elasticsearch/sink/ElasticsearchSink.java | 38 ++----------------- .../sink/ElasticsearchWriter.java | 10 ++--- .../elasticsearch/sink/RequestIndexer.java | 4 +- .../table/ElasticsearchDynamicSink.java | 4 +- .../table/RowElasticsearchEmitter.java | 2 +- .../sink/ElasticsearchWriterITCase.java | 4 +- .../elasticsearch/sink/TestEmitter.java | 2 +- .../ElasticsearchDynamicSinkBaseITCase.java | 8 ++-- ...asticsearchDynamicSinkFactoryBaseTest.java | 6 +-- 10 files changed, 24 insertions(+), 56 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java index b3815afe..950f8627 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.Function; -import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.elasticsearch.action.ActionRequest; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index 4a47fe6d..30ee238b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -19,18 +19,14 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.connector.sink.Committer; -import org.apache.flink.api.connector.sink.GlobalCommitter; -import org.apache.flink.api.connector.sink.Sink; -import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.connector.base.DeliveryGuarantee; -import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.http.HttpHost; import java.io.IOException; import java.util.List; -import java.util.Optional; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -55,7 +51,7 @@ * @see ElasticsearchSinkBuilderBase on how to construct a ElasticsearchSink */ @PublicEvolving -public class ElasticsearchSink implements Sink { +public class ElasticsearchSink implements Sink { private final List hosts; private final ElasticsearchEmitter emitter; @@ -81,8 +77,7 @@ public class ElasticsearchSink implements Sink { } @Override - public SinkWriter createWriter(InitContext context, List states) - throws IOException { + public SinkWriter createWriter(InitContext context) throws IOException { return new ElasticsearchWriter<>( hosts, emitter, @@ -93,29 +88,4 @@ public SinkWriter createWriter(InitContext context, List s context.metricGroup(), context.getMailboxExecutor()); } - - @Override - public Optional> getWriterStateSerializer() { - return Optional.empty(); - } - - @Override - public Optional> createCommitter() throws IOException { - return Optional.empty(); - } - - @Override - public Optional> createGlobalCommitter() throws IOException { - return Optional.empty(); - } - - @Override - public Optional> getCommittableSerializer() { - return Optional.empty(); - } - - @Override - public Optional> getGlobalCommittableSerializer() { - return Optional.empty(); - } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 53723bc7..7ea3dd87 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.util.FlinkRuntimeException; @@ -47,13 +47,12 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Collections; import java.util.List; import static org.apache.flink.util.ExceptionUtils.firstOrSuppressed; import static org.apache.flink.util.Preconditions.checkNotNull; -class ElasticsearchWriter implements SinkWriter { +class ElasticsearchWriter implements SinkWriter { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchWriter.class); @@ -125,15 +124,14 @@ public void write(IN element, Context context) throws IOException, InterruptedEx } @Override - public List prepareCommit(boolean flush) throws IOException, InterruptedException { + public void flush(boolean endOfInput) throws IOException, InterruptedException { checkpointInProgress = true; - while (pendingActions != 0 && (flushOnCheckpoint || flush)) { + while (pendingActions != 0 && (flushOnCheckpoint || endOfInput)) { bulkProcessor.flush(); LOG.info("Waiting for the response of {} pending actions.", pendingActions); mailboxExecutor.yield(); } checkpointInProgress = false; - return Collections.emptyList(); } @VisibleForTesting diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java index 2bb8732f..830fb6a3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java @@ -18,7 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; -import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; @@ -28,7 +28,7 @@ * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare them * for sending to an Elasticsearch cluster. */ -@Internal +@PublicEvolving public interface RequestIndexer { /** * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index fbab8b78..975db5c1 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -26,7 +26,7 @@ import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.connector.sink.SinkV2Provider; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; @@ -157,7 +157,7 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { builder.setSocketTimeout((int) config.getSocketTimeout().get().getSeconds()); } - return SinkProvider.of(builder.build(), config.getParallelism().orElse(null)); + return SinkV2Provider.of(builder.build(), config.getParallelism().orElse(null)); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java index 8b0fd07a..e90e0bca 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java @@ -19,7 +19,7 @@ package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.connector.elasticsearch.sink.ElasticsearchEmitter; import org.apache.flink.connector.elasticsearch.sink.RequestIndexer; import org.apache.flink.table.api.TableException; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 4fea99ee..7b940668 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -113,7 +113,7 @@ void testWriteOnBulkFlush() throws Exception { writer.write(Tuple2.of(4, buildMessage(4)), null); // Ignore flush on checkpoint - writer.prepareCommit(false); + writer.flush(false); context.assertThatIdsAreNotWritten(index, 1, 2, 3, 4); @@ -166,7 +166,7 @@ void testWriteOnCheckpoint() throws Exception { context.assertThatIdsAreNotWritten(index, 1, 2, 3); // Trigger flush - writer.prepareCommit(false); + writer.flush(false); context.assertThatIdsAreWritten(index, 1, 2, 3); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java index 73b512d7..49943b23 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -17,7 +17,7 @@ package org.apache.flink.connector.elasticsearch.sink; -import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.elasticsearch.action.index.IndexRequest; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index a6ab6bea..192d36cd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -19,7 +19,7 @@ package org.apache.flink.connector.elasticsearch.table; import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; @@ -29,7 +29,7 @@ import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.connector.sink.SinkV2Provider; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; @@ -117,8 +117,8 @@ public void testWritingDocuments() throws Exception { getPrefilledTestContext(index).withSchema(schema).build()) .getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); - final SinkProvider sinkProvider = (SinkProvider) runtimeProvider; - final Sink sink = sinkProvider.createSink(); + final SinkV2Provider sinkProvider = (SinkV2Provider) runtimeProvider; + final Sink sink = sinkProvider.createSink(); StreamExecutionEnvironment environment = StreamExecutionEnvironment.getExecutionEnvironment(); environment.setParallelism(4); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java index c151e719..7a1e04f4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -26,7 +26,7 @@ import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.SinkProvider; +import org.apache.flink.table.connector.sink.SinkV2Provider; import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Assertions; @@ -244,8 +244,8 @@ public void testSinkParallelism() { .build()); assertThat(sink).isInstanceOf(ElasticsearchDynamicSink.class); ElasticsearchDynamicSink esSink = (ElasticsearchDynamicSink) sink; - SinkProvider provider = - (SinkProvider) esSink.getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); + SinkV2Provider provider = + (SinkV2Provider) esSink.getSinkRuntimeProvider(new ElasticsearchUtil.MockContext()); assertThat(2).isEqualTo(provider.getParallelism().get()); } } From e747a9bba1fc4f91b2a997f801913d9cd6ebacf7 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Mon, 14 Feb 2022 16:36:57 +0100 Subject: [PATCH 187/207] [hotfix] Fix JUnit 5 mini cluster arch unit rules --- .../archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e | 1 + .../archunit-violations/stored.rules | 3 ++- .../archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe | 3 +++ .../archunit-violations/stored.rules | 3 ++- .../archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 | 3 +++ .../archunit-violations/stored.rules | 3 ++- 6 files changed, 13 insertions(+), 3 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e create mode 100644 flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe create mode 100644 flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e new file mode 100644 index 00000000..249f3616 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e @@ -0,0 +1 @@ +org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules index ffb1cfd9..ef334dab 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules @@ -1,4 +1,5 @@ # -#Thu Feb 03 15:07:30 CET 2022 +#Wed Feb 16 08:38:38 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=1a2e2fb0-c5ed-438f-996f-10fa86cf006e classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f527dc38-29ae-4dfd-b8f6-9ee4628fec79 classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=1dc63ea4-0c1a-473b-b904-a5141706cac1 diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe new file mode 100644 index 00000000..6adc4f9d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe @@ -0,0 +1,3 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules index 5c13f4d5..2ca6654d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules @@ -1,4 +1,5 @@ # -#Thu Feb 03 14:58:57 CET 2022 +#Wed Feb 16 08:40:40 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=fa5c33af-641c-4a62-bd5b-ffe9885e49fe classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=cace802d-2fe7-4eef-aef1-8ba8e335f854 classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=2626cbc6-34aa-46c3-a57a-a4f934cc4e9e diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 new file mode 100644 index 00000000..75c5a5cc --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 @@ -0,0 +1,3 @@ +org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules index f38293c8..04267dcc 100644 --- a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules @@ -1,4 +1,5 @@ # -#Thu Feb 03 15:06:52 CET 2022 +#Wed Feb 16 08:41:40 CET 2022 +classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=075c1eb7-67e6-45f3-91d4-d3c9fde9df77 classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f1068e32-05ee-4824-8b9e-953d6fd42958 classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=6802607d-5f0d-4f36-bb90-9566760ecb89 From a81cf115e5494f7283ae57c560be29dcfcfc6f5a Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Tue, 22 Feb 2022 21:44:42 +0800 Subject: [PATCH 188/207] [FLINK-26294][test] Using fixed description for ArchUnit ITCaseRules --- ...f-10fa86cf006e => dd583797-83e1-414c-a38d-330773978813} | 0 ...4-a5141706cac1 => de342dd1-c974-42c9-8f64-ef182ba8c56d} | 0 .../f527dc38-29ae-4dfd-b8f6-9ee4628fec79 | 1 - .../archunit-violations/stored.rules | 7 +++---- ...a-a4f934cc4e9e => 25e52d29-fa7e-42fa-a571-b5c76235df52} | 0 .../cace802d-2fe7-4eef-aef1-8ba8e335f854 | 3 --- ...b-ffe9885e49fe => db3972e4-f3a3-45b2-9643-27cba0cef09d} | 0 .../archunit-violations/stored.rules | 7 +++---- ...0-9566760ecb89 => 1af7baaa-05dc-452a-9de7-653c8b3b324f} | 0 ...4-d3c9fde9df77 => e1f30f33-c61c-4707-8c78-a3a80479564e} | 0 .../f1068e32-05ee-4824-8b9e-953d6fd42958 | 3 --- .../archunit-violations/stored.rules | 7 +++---- 12 files changed, 9 insertions(+), 19 deletions(-) rename flink-connectors/flink-connector-elasticsearch-base/archunit-violations/{1a2e2fb0-c5ed-438f-996f-10fa86cf006e => dd583797-83e1-414c-a38d-330773978813} (100%) rename flink-connectors/flink-connector-elasticsearch-base/archunit-violations/{1dc63ea4-0c1a-473b-b904-a5141706cac1 => de342dd1-c974-42c9-8f64-ef182ba8c56d} (100%) delete mode 100644 flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 rename flink-connectors/flink-connector-elasticsearch6/archunit-violations/{2626cbc6-34aa-46c3-a57a-a4f934cc4e9e => 25e52d29-fa7e-42fa-a571-b5c76235df52} (100%) delete mode 100644 flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 rename flink-connectors/flink-connector-elasticsearch6/archunit-violations/{fa5c33af-641c-4a62-bd5b-ffe9885e49fe => db3972e4-f3a3-45b2-9643-27cba0cef09d} (100%) rename flink-connectors/flink-connector-elasticsearch7/archunit-violations/{6802607d-5f0d-4f36-bb90-9566760ecb89 => 1af7baaa-05dc-452a-9de7-653c8b3b324f} (100%) rename flink-connectors/flink-connector-elasticsearch7/archunit-violations/{075c1eb7-67e6-45f3-91d4-d3c9fde9df77 => e1f30f33-c61c-4707-8c78-a3a80479564e} (100%) delete mode 100644 flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1a2e2fb0-c5ed-438f-996f-10fa86cf006e rename to flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1dc63ea4-0c1a-473b-b904-a5141706cac1 b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/archunit-violations/1dc63ea4-0c1a-473b-b904-a5141706cac1 rename to flink-connectors/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 deleted file mode 100644 index e9fa01bd..00000000 --- a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/f527dc38-29ae-4dfd-b8f6-9ee4628fec79 +++ /dev/null @@ -1 +0,0 @@ -org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules index ef334dab..76f9416d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules @@ -1,5 +1,4 @@ # -#Wed Feb 16 08:38:38 CET 2022 -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=1a2e2fb0-c5ed-438f-996f-10fa86cf006e -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f527dc38-29ae-4dfd-b8f6-9ee4628fec79 -classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=1dc63ea4-0c1a-473b-b904-a5141706cac1 +#Tue Feb 22 12:16:59 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=de342dd1-c974-42c9-8f64-ef182ba8c56d +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=dd583797-83e1-414c-a38d-330773978813 diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/2626cbc6-34aa-46c3-a57a-a4f934cc4e9e b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/archunit-violations/2626cbc6-34aa-46c3-a57a-a4f934cc4e9e rename to flink-connectors/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 deleted file mode 100644 index bddadc6d..00000000 --- a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/cace802d-2fe7-4eef-aef1-8ba8e335f854 +++ /dev/null @@ -1,3 +0,0 @@ -org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/archunit-violations/fa5c33af-641c-4a62-bd5b-ffe9885e49fe rename to flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules index 2ca6654d..10c711f0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules @@ -1,5 +1,4 @@ # -#Wed Feb 16 08:40:40 CET 2022 -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=fa5c33af-641c-4a62-bd5b-ffe9885e49fe -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=cace802d-2fe7-4eef-aef1-8ba8e335f854 -classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=2626cbc6-34aa-46c3-a57a-a4f934cc4e9e +#Tue Feb 22 12:17:08 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=25e52d29-fa7e-42fa-a571-b5c76235df52 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=db3972e4-f3a3-45b2-9643-27cba0cef09d diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/6802607d-5f0d-4f36-bb90-9566760ecb89 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/archunit-violations/6802607d-5f0d-4f36-bb90-9566760ecb89 rename to flink-connectors/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/archunit-violations/075c1eb7-67e6-45f3-91d4-d3c9fde9df77 rename to flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 deleted file mode 100644 index ab17befe..00000000 --- a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/f1068e32-05ee-4824-8b9e-953d6fd42958 +++ /dev/null @@ -1,3 +0,0 @@ -org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase does not satisfy: contain any fields that are public, static, and of type MiniClusterExtension and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules index 04267dcc..f2bb6a71 100644 --- a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules @@ -1,5 +1,4 @@ # -#Wed Feb 16 08:41:40 CET 2022 -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ is\ is\ assignable\ to\ MiniClusterExtension\ and\ public\ and\ static\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=075c1eb7-67e6-45f3-91d4-d3c9fde9df77 -classes\ that\ are\ Java\ classes\ and\ have\ simple\ name\ ending\ with\ 'ITCase'\ and\ are\ top\ level\ classes\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterExtension\ and\ final\ and\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ AllCallbackWrapper\ and\ final\ and\ annotated\ with\ @RegisterExtension\ or\ contain\ any\ fields\ that\ are\ public,\ static,\ and\ of\ type\ MiniClusterWithClientResource\ and\ final\ and\ annotated\ with\ @ClassRule\ or\ contain\ any\ fields\ that\ is\ of\ type\ MiniClusterWithClientResource\ and\ public\ and\ final\ and\ not\ static\ and\ annotated\ with\ @Rule=f1068e32-05ee-4824-8b9e-953d6fd42958 -classes\ that\ are\ Java\ classes\ and\ are\ assignable\ to\ org.apache.flink.test.util.AbstractTestBase\ and\ do\ not\ have\ modifier\ ABSTRACT\ should\ have\ simple\ name\ ending\ with\ 'ITCase'=6802607d-5f0d-4f36-bb90-9566760ecb89 +#Tue Feb 22 12:17:16 CET 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=1af7baaa-05dc-452a-9de7-653c8b3b324f +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=e1f30f33-c61c-4707-8c78-a3a80479564e From 86b3dd1ab76043fe76552c1cf6e2d29e6fca8614 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Tue, 22 Feb 2022 14:53:29 +0100 Subject: [PATCH 189/207] [hotfix][tests] Add instructions for updating archunit stores --- .../archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d | 3 +-- .../archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d index 6adc4f9d..2520fe86 100644 --- a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d @@ -1,3 +1,2 @@ org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file +org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e index 75c5a5cc..854c270e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e @@ -1,3 +1,2 @@ org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file +org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file From 83dcc950b3ea73136cae980d8de5d56ef93ecd11 Mon Sep 17 00:00:00 2001 From: jinfeng Date: Thu, 9 Dec 2021 20:36:48 +0800 Subject: [PATCH 190/207] [FLINK-24571][connectors/elasticsearch] Supports a system time function(now() and current_timestamp) in index pattern --- .../table/ElasticsearchDynamicSink.java | 24 ++++++- .../ElasticsearchDynamicSinkFactoryBase.java | 16 ++++- .../table/IndexGeneratorFactory.java | 45 ++++++++++-- .../ElasticsearchDynamicSinkBaseITCase.java | 58 ++++++++++++++++ ...asticsearchDynamicSinkFactoryBaseTest.java | 23 +++++++ .../table/IndexGeneratorTest.java | 69 +++++++++++++++++++ 6 files changed, 227 insertions(+), 8 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java index 975db5c1..0fd389bd 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java @@ -23,6 +23,7 @@ import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink; import org.apache.flink.connector.elasticsearch.sink.ElasticsearchSinkBuilderBase; import org.apache.flink.connector.elasticsearch.sink.FlushBackoffType; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.EncodingFormat; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -37,6 +38,7 @@ import javax.annotation.Nullable; +import java.time.ZoneId; import java.util.List; import java.util.Objects; import java.util.function.Function; @@ -54,10 +56,12 @@ class ElasticsearchDynamicSink implements DynamicTableSink { final DataType physicalRowDataType; final List primaryKeyLogicalTypesWithIndex; final ElasticsearchConfiguration config; + final ZoneId localTimeZoneId; final String summaryString; final ElasticsearchSinkBuilderSupplier builderSupplier; @Nullable final String documentType; + final boolean isDynamicIndexWithSystemTime; ElasticsearchDynamicSink( EncodingFormat> format, @@ -66,7 +70,8 @@ class ElasticsearchDynamicSink implements DynamicTableSink { DataType physicalRowDataType, String summaryString, ElasticsearchSinkBuilderSupplier builderSupplier, - @Nullable String documentType) { + @Nullable String documentType, + ZoneId localTimeZoneId) { this.format = checkNotNull(format); this.physicalRowDataType = checkNotNull(physicalRowDataType); this.primaryKeyLogicalTypesWithIndex = checkNotNull(primaryKeyLogicalTypesWithIndex); @@ -74,6 +79,13 @@ class ElasticsearchDynamicSink implements DynamicTableSink { this.summaryString = checkNotNull(summaryString); this.builderSupplier = checkNotNull(builderSupplier); this.documentType = documentType; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + } + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); } Function createKeyExtractor() { @@ -85,7 +97,8 @@ IndexGenerator createIndexGenerator() { return IndexGeneratorFactory.createIndexGenerator( config.getIndex(), DataType.getFieldNames(physicalRowDataType), - DataType.getFieldDataTypes(physicalRowDataType)); + DataType.getFieldDataTypes(physicalRowDataType), + localTimeZoneId); } @Override @@ -96,6 +109,10 @@ public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { builder.addContainedKind(kind); } } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } return builder.build(); } @@ -169,7 +186,8 @@ public DynamicTableSink copy() { physicalRowDataType, summaryString, builderSupplier, - documentType); + documentType, + localTimeZoneId); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java index af358883..ed5e7f73 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java @@ -21,7 +21,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.Projection; @@ -37,6 +39,7 @@ import javax.annotation.Nullable; +import java.time.ZoneId; import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -106,13 +109,24 @@ public DynamicTableSink createDynamicTableSink(Context context) { context.getPhysicalRowDataType(), capitalize(factoryIdentifier), sinkBuilderSupplier, - getDocumentType(config)); + getDocumentType(config), + getLocalTimeZoneId(context.getConfiguration())); } ElasticsearchConfiguration getConfiguration(FactoryUtil.TableFactoryHelper helper) { return new ElasticsearchConfiguration(helper.getOptions()); } + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + void validateConfiguration(ElasticsearchConfiguration config) { config.getHosts(); // validate hosts validate( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java index baf0e04f..ec2a0069 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java @@ -30,7 +30,9 @@ import java.io.Serializable; import java.time.LocalDate; +import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.ArrayList; @@ -63,30 +65,57 @@ final class IndexGeneratorFactory { private IndexGeneratorFactory() {} public static IndexGenerator createIndexGenerator( - String index, List fieldNames, List dataTypes) { + String index, + List fieldNames, + List dataTypes, + ZoneId localTimeZoneId) { final IndexHelper indexHelper = new IndexHelper(); if (indexHelper.checkIsDynamicIndex(index)) { return createRuntimeIndexGenerator( index, fieldNames.toArray(new String[0]), dataTypes.toArray(new DataType[0]), - indexHelper); + indexHelper, + localTimeZoneId); } else { return new StaticIndexGenerator(index); } } + public static IndexGenerator createIndexGenerator( + String index, List fieldNames, List dataTypes) { + return createIndexGenerator(index, fieldNames, dataTypes, ZoneId.systemDefault()); + } + interface DynamicFormatter extends Serializable { String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); } private static IndexGenerator createRuntimeIndexGenerator( - String index, String[] fieldNames, DataType[] fieldTypes, IndexHelper indexHelper) { + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper, + ZoneId localTimeZoneId) { final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); final String indexSuffix = index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) { + final String dateTimeFormat = + indexHelper.extractDateFormat( + index, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + return indexPrefix + .concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter)) + .concat(indexSuffix); + } + }; + } + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); final int indexFieldPos = indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); @@ -172,10 +201,13 @@ private static DynamicFormatter createFormatFunction( * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field * type ans parse index format from pattern. */ - private static class IndexHelper { + static class IndexHelper { private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); private static final Pattern dynamicIndexTimeExtractPattern = Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final Pattern dynamicIndexSystemTimeExtractPattern = + Pattern.compile( + ".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*"); private static final List supportedTypes = new ArrayList<>(); private static final Map defaultFormats = new HashMap<>(); @@ -240,6 +272,11 @@ boolean checkIsDynamicIndexWithFormat(String index) { return dynamicIndexTimeExtractPattern.matcher(index).matches(); } + /** Check generate dynamic index is from system time or not. */ + boolean checkIsDynamicIndexWithSystemTimeFormat(String index) { + return dynamicIndexSystemTimeExtractPattern.matcher(index).matches(); + } + /** Extract dynamic index pattern string from index pattern string. */ String extractDynamicIndexPatternStr(String index) { int start = index.indexOf("{"); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index 192d36cd..21ad5fd2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -38,8 +38,10 @@ import org.apache.flink.util.TestLoggerExtension; import org.apache.http.HttpHost; +import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHits; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -50,6 +52,8 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -299,4 +303,58 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { expectedMap.put("b", "2012-12-12 12:12:12"); Assertions.assertEquals(response, expectedMap); } + + @Test + public void testWritingDocumentsWithDynamicIndexFromSystemTime() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); + tableEnvironment + .getConfig() + .getConfiguration() + .setString("table.local-time-zone", "Asia/Shanghai"); + + String dynamicIndex1 = + "dynamic-index-" + + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai"))) + + "_index"; + String index = "dynamic-index-{now()|yyyy-MM-dd}_index"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + getConnectorSql(index) + + ")"); + String dynamicIndex2 = + "dynamic-index-" + + dateTimeFormatter.format(LocalDateTime.now(ZoneId.of("Asia/Shanghai"))) + + "_index"; + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + RestHighLevelClient client = getClient(); + + Map response; + try { + response = makeGetRequest(client, dynamicIndex1, "1"); + } catch (ElasticsearchStatusException e) { + if (e.status() == RestStatus.NOT_FOUND) { + response = makeGetRequest(client, dynamicIndex2, "1"); + } else { + throw e; + } + } + + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + Assertions.assertEquals(response, expectedMap); + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java index 7a1e04f4..8fa0a498 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java @@ -25,8 +25,10 @@ import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.types.RowKind; import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Assertions; @@ -234,6 +236,27 @@ public void validateWrongCredential() { .build())); } + @Test + public void validateDynamicIndexOnChangelogStream() { + ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); + DynamicTableSink sink = + sinkFactory.createDynamicTableSink( + createPrefilledTestContext() + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION.key(), + "dynamic-index-{now()|yyyy-MM-dd}_index") + .build()); + + ChangelogMode changelogMode = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.DELETE) + .addContainedKind(RowKind.INSERT) + .build(); + assertValidationException( + "Dynamic indexing based on system time only works on append only stream.", + () -> sink.getChangelogMode(changelogMode)); + } + @Test public void testSinkParallelism() { ElasticsearchDynamicSinkFactoryBase sinkFactory = createSinkFactory(); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java index de2aad58..7840bfa8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java @@ -35,6 +35,7 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.format.DateTimeFormatter; import java.time.temporal.UnsupportedTemporalTypeException; import java.util.Arrays; import java.util.List; @@ -181,6 +182,74 @@ public void testDynamicIndexDefaultFormat() { Assertions.assertEquals("my-index-12_13_14", indexGenerator.generate(rows.get(1))); } + @Test + public void testDynamicIndexFromSystemTime() { + List supportedUseCases = + Arrays.asList( + "now()", + "NOW()", + "now( )", + "NOW(\t)", + "\t NOW( ) \t", + "current_timestamp", + "CURRENT_TIMESTAMP", + "\tcurrent_timestamp\t", + " current_timestamp "); + + supportedUseCases.stream() + .forEach( + f -> { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + fieldNames, + dataTypes); + indexGenerator.open(); + // The date may change during the running of the unit test. + // Generate expected index-name based on the current time + // before and after calling the generate method. + String expectedIndex1 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + String actualIndex = indexGenerator.generate(rows.get(1)); + String expectedIndex2 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + Assertions.assertTrue( + actualIndex.equals(expectedIndex1) + || actualIndex.equals(expectedIndex2)); + }); + + List invalidUseCases = + Arrays.asList( + "now", + "now(", + "NOW", + "NOW)", + "current_timestamp()", + "CURRENT_TIMESTAMP()", + "CURRENT_timestamp"); + invalidUseCases.stream() + .forEach( + f -> { + String expectedExceptionMsg = + String.format( + "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}'," + + " please check the field name.", + f, f); + try { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + fieldNames, + dataTypes); + indexGenerator.open(); + } catch (TableException e) { + Assertions.assertEquals(expectedExceptionMsg, e.getMessage()); + } + }); + } + @Test public void testGeneralDynamicIndex() { IndexGenerator indexGenerator = From e3caa30c874152e0458831f51c5b671070269276 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Tue, 22 Feb 2022 16:07:05 +0100 Subject: [PATCH 191/207] [FLINK-26252][tests] Refactor MiniClusterExtension to support JUnit 5 parallel tests --- .../dd583797-83e1-414c-a38d-330773978813 | 7 ++++++- .../db3972e4-f3a3-45b2-9643-27cba0cef09d | 14 ++++++++++++-- .../e1f30f33-c61c-4707-8c78-a3a80479564e | 14 ++++++++++++-- 3 files changed, 30 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 index 249f3616..a215b58b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 +++ b/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 @@ -1 +1,6 @@ -org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file +org.apache.flink.connector.elasticsearch.sink.ElasticsearchWriterITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d index 2520fe86..341409ac 100644 --- a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d +++ b/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d @@ -1,2 +1,12 @@ -org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file +org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e index 854c270e..e6b90b93 100644 --- a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e +++ b/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e @@ -1,2 +1,12 @@ -org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule -org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: contain any fields that is is assignable to MiniClusterExtension and public and static and final and contain any fields that are public, static, and of type AllCallbackWrapper and final and annotated with @RegisterExtension or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file +org.apache.flink.connector.elasticsearch.sink.Elasticsearch7SinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file From cb8b580129d2939c3067b7da4ecd2b6e23d7a1ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 22 Feb 2022 10:04:35 +0100 Subject: [PATCH 192/207] [FLINK-26191][connectors/elasticsearch] Revert Elasticsearch dependencies to 7.10.2 to keep Apache 2 license --- .../pom.xml | 2 +- .../sink/ElasticsearchWriterITCase.java | 2 +- .../ElasticsearchSinkBaseTest.java | 4 +- .../testutils/ElasticsearchResource.java | 2 +- .../flink-connector-elasticsearch7/pom.xml | 2 +- .../sink/Elasticsearch7SinkBuilder.java | 2 +- .../Elasticsearch7ApiCallBridge.java | 2 +- .../src/main/resources/META-INF/NOTICE | 58 +++++++++---------- 8 files changed, 37 insertions(+), 37 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index b42f97c0..cd2c6c59 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -37,7 +37,7 @@ under the License. - 7.15.2 + 7.10.2 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index 7b940668..d0132a0b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -45,7 +45,7 @@ import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.unit.TimeValue; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 409c6ec2..98c40930 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -601,14 +601,14 @@ public Object answer(InvocationOnMock invocationOnMock) if (mockItemFailure == null) { // the mock response for the item is success mockResponses[i] = - BulkItemResponse.success( + new BulkItemResponse( i, DocWriteRequest.OpType.INDEX, mock(DocWriteResponse.class)); } else { // the mock response for the item is failure mockResponses[i] = - BulkItemResponse.failure( + new BulkItemResponse( i, DocWriteRequest.OpType.INDEX, new BulkItemResponse.Failure( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java index e5716dcf..cd11b70d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java @@ -27,7 +27,7 @@ import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; import org.elasticsearch.client.ClusterAdminClient; -import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.unit.TimeValue; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 74870223..1c1a1951 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -37,7 +37,7 @@ under the License. - 7.15.2 + 7.10.2 diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java index e981b942..3a478273 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -28,7 +28,7 @@ import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.unit.TimeValue; /** * Builder to construct an Elasticsearch 7 compatible {@link ElasticsearchSink}. diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java index 9440097a..b076a30e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java @@ -31,7 +31,7 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.unit.TimeValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index fe1ccd52..fa04fce1 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -20,33 +20,33 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.httpcomponents:httpclient:4.5.13 - org.apache.httpcomponents:httpcore:4.4.14 - org.apache.httpcomponents:httpcore-nio:4.4.12 -- org.apache.lucene:lucene-analyzers-common:8.9.0 -- org.apache.lucene:lucene-backward-codecs:8.9.0 -- org.apache.lucene:lucene-core:8.9.0 -- org.apache.lucene:lucene-grouping:8.9.0 -- org.apache.lucene:lucene-highlighter:8.9.0 -- org.apache.lucene:lucene-join:8.9.0 -- org.apache.lucene:lucene-memory:8.9.0 -- org.apache.lucene:lucene-misc:8.9.0 -- org.apache.lucene:lucene-queries:8.9.0 -- org.apache.lucene:lucene-queryparser:8.9.0 -- org.apache.lucene:lucene-sandbox:8.9.0 -- org.apache.lucene:lucene-spatial:8.9.0 -- org.apache.lucene:lucene-spatial-extras:8.9.0 -- org.apache.lucene:lucene-spatial3d:8.9.0 -- org.apache.lucene:lucene-suggest:8.9.0 -- org.elasticsearch:elasticsearch:7.15.2 -- org.elasticsearch:elasticsearch-cli:7.15.2 -- org.elasticsearch:elasticsearch-core:7.15.2 -- org.elasticsearch:elasticsearch-geo:7.15.2 -- org.elasticsearch:elasticsearch-secure-sm:7.15.2 -- org.elasticsearch:elasticsearch-x-content:7.15.2 -- org.elasticsearch:elasticsearch-plugin-classloader:7.15.2 -- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.15.2 -- org.elasticsearch.client:elasticsearch-rest-client:7.15.2 -- org.elasticsearch.plugin:aggs-matrix-stats-client:7.15.2 -- org.elasticsearch.plugin:lang-mustache-client:7.15.2 -- org.elasticsearch.plugin:mapper-extras-client:7.15.2 -- org.elasticsearch.plugin:parent-join-client:7.15.2 -- org.elasticsearch.plugin:rank-eval-client:7.15.2 +- org.apache.lucene:lucene-analyzers-common:8.7.0 +- org.apache.lucene:lucene-backward-codecs:8.7.0 +- org.apache.lucene:lucene-core:8.7.0 +- org.apache.lucene:lucene-grouping:8.7.0 +- org.apache.lucene:lucene-highlighter:8.7.0 +- org.apache.lucene:lucene-join:8.7.0 +- org.apache.lucene:lucene-memory:8.7.0 +- org.apache.lucene:lucene-misc:8.7.0 +- org.apache.lucene:lucene-queries:8.7.0 +- org.apache.lucene:lucene-queryparser:8.7.0 +- org.apache.lucene:lucene-sandbox:8.7.0 +- org.apache.lucene:lucene-spatial:8.7.0 +- org.apache.lucene:lucene-spatial-extras:8.7.0 +- org.apache.lucene:lucene-spatial3d:8.7.0 +- org.apache.lucene:lucene-suggest:8.7.0 +- org.elasticsearch:elasticsearch:7.10.2 +- org.elasticsearch:elasticsearch-cli:7.10.2 +- org.elasticsearch:elasticsearch-core:7.10.2 +- org.elasticsearch:elasticsearch-geo:7.10.2 +- org.elasticsearch:elasticsearch-secure-sm:7.10.2 +- org.elasticsearch:elasticsearch-x-content:7.10.2 +- org.elasticsearch:elasticsearch-plugin-classloader:7.10.2 +- org.elasticsearch.client:elasticsearch-rest-high-level-client:7.10.2 +- org.elasticsearch.client:elasticsearch-rest-client:7.10.2 +- org.elasticsearch.plugin:aggs-matrix-stats-client:7.10.2 +- org.elasticsearch.plugin:lang-mustache-client:7.10.2 +- org.elasticsearch.plugin:mapper-extras-client:7.10.2 +- org.elasticsearch.plugin:parent-join-client:7.10.2 +- org.elasticsearch.plugin:rank-eval-client:7.10.2 - org.lz4:lz4-java:1.8.0 From abd5a7944bcf5a5ad6a58b4f23ae5ae019d6bb6f Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Mon, 14 Mar 2022 16:52:19 +0100 Subject: [PATCH 193/207] [FLINK-25927][connectors][build] Consistent flink-connector-base usage flink-connector-base was previously inconsistently used in connectors (directly shaded in some and transitively pulled in via flink-connector-files which was itself shaded in the table uber jar). FLINK-24687 moved flink-connector-files out from the flink-table uber jar. This commit implements a combined approach for ensuring a smooth transition for both Flink users and for external connector developers, as outlined in this: - all internal Flink connectors that depend on flink-connector-base now shade and relocate it - for compatibility, until external developers implement the same change, flink-connector-base is also included into flink-dist --- .../flink-connector-elasticsearch6/pom.xml | 27 +++++++++++++++++++ .../flink-connector-elasticsearch7/pom.xml | 27 +++++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 6a5b5e86..92bc8f12 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -199,6 +199,33 @@ under the License. 1 + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-base + org.apache.flink:flink-connector-elasticsearch-base + + + + + org.apache.flink.connector.base + org.apache.flink.connector.elasticsearch6.shaded.org.apache.flink.connector.base + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 1c1a1951..4f01962f 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -196,6 +196,33 @@ under the License. 1 + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + package + + shade + + + + + org.apache.flink:flink-connector-base + org.apache.flink:flink-connector-elasticsearch-base + + + + + org.apache.flink.connector.base + org.apache.flink.connector.elasticsearch7.shaded.org.apache.flink.connector.base + + + + + + From 9fad48aaed05b2651c7e85789c959497e23fce7d Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Tue, 15 Mar 2022 01:27:55 +0800 Subject: [PATCH 194/207] Update version to 1.16-SNAPSHOT --- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index cd2c6c59..a7417663 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 92bc8f12..353fbe47 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 4f01962f..a8e84160 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 6aaaf4ac..bf31dcf1 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 489e9ea3..a2a5c730 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.16-SNAPSHOT .. From eb5789cf50f190aa8d274234e4bf777c60c1c5dd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 10 Mar 2022 09:48:58 +0100 Subject: [PATCH 195/207] [FLINK-26281][connectors/elasticsearch] setting default delivery guarantee to AT_LEAST_ONCE --- .../connector/elasticsearch/sink/ElasticsearchSink.java | 6 ++++++ .../elasticsearch/sink/ElasticsearchSinkBuilderBase.java | 2 +- .../elasticsearch/table/ElasticsearchConnectorOptions.java | 2 +- .../sink/ElasticsearchSinkBuilderBaseTest.java | 7 +++++++ .../elasticsearch/sink/Elasticsearch6SinkBuilder.java | 1 - .../elasticsearch/sink/Elasticsearch7SinkBuilder.java | 1 - 6 files changed, 15 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java index 30ee238b..efe6dc24 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.connector.base.DeliveryGuarantee; @@ -88,4 +89,9 @@ public SinkWriter createWriter(InitContext context) throws IOException { context.metricGroup(), context.getMailboxExecutor()); } + + @VisibleForTesting + DeliveryGuarantee getDeliveryGuarantee() { + return deliveryGuarantee; + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java index fe64c949..8543255d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java @@ -48,7 +48,7 @@ public abstract class ElasticsearchSinkBuilderBase< private FlushBackoffType bulkFlushBackoffType = FlushBackoffType.NONE; private int bulkFlushBackoffRetries = -1; private long bulkFlushBackOffDelay = -1; - private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.NONE; + private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE; private List hosts; protected ElasticsearchEmitter emitter; private String username; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java index 672f0727..10ea0ae2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -143,6 +143,6 @@ public class ElasticsearchConnectorOptions { public static final ConfigOption DELIVERY_GUARANTEE_OPTION = ConfigOptions.key("sink.delivery-guarantee") .enumType(DeliveryGuarantee.class) - .defaultValue(DeliveryGuarantee.NONE) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) .withDescription("Optional delivery guarantee when committing."); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java index 695f85c1..4d1890fe 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java @@ -29,6 +29,7 @@ import java.util.stream.Stream; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -56,6 +57,12 @@ Stream testValidBuilders() { builder -> assertDoesNotThrow(builder::build)); } + @Test + void testDefaultDeliveryGuarantee() { + assertThat(createMinimalBuilder().build().getDeliveryGuarantee()) + .isEqualTo(DeliveryGuarantee.AT_LEAST_ONCE); + } + @Test void testThrowIfExactlyOnceConfigured() { assertThrows( diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java index 34c098b8..c90ccaca 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java @@ -46,7 +46,6 @@ * .source(element.f1) * ); * }) - * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) * .build(); * } * diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java index 3a478273..567f7740 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java @@ -46,7 +46,6 @@ * .source(element.f1) * ); * }) - * .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) * .build(); * } * From 727731176cefd908993fcaeabecbc38720b63bca Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 17 Mar 2022 11:20:16 +0100 Subject: [PATCH 196/207] [FLINK-26701][connectors] Revert FLINK-25927 apart from adding connector-base to flink-dist --- .../flink-connector-elasticsearch6/pom.xml | 27 ------------------- .../flink-connector-elasticsearch7/pom.xml | 27 ------------------- 2 files changed, 54 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 353fbe47..3af8f666 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -199,33 +199,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch6.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index a8e84160..3c4bb4f7 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -196,33 +196,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch7.shaded.org.apache.flink.connector.base - - - - - - From 629f865d8980ba61f0fe65ec143b862664af3efc Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Thu, 17 Mar 2022 11:53:21 +0100 Subject: [PATCH 197/207] [hotfix][elasticsearch] Make TestEmitter#createIndexRequest private --- .../apache/flink/connector/elasticsearch/sink/TestEmitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java index 49943b23..2cfc5eb4 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -58,7 +58,7 @@ public void emit( indexer.add(createIndexRequest(element)); } - public IndexRequest createIndexRequest(Tuple2 element) { + private IndexRequest createIndexRequest(Tuple2 element) { Map document = new HashMap<>(); document.put(dataFieldName, element.f1); try { From 7ea06d899335a469d093de6077de90ef32da18b4 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Mon, 14 Mar 2022 15:24:35 +0100 Subject: [PATCH 198/207] [FLINK-26633][elasticsearch] Add recordSend metric to elasticsearch sink --- .../sink/ElasticsearchWriter.java | 11 ++- .../sink/ElasticsearchWriterITCase.java | 80 ++++++++++++++++++- 2 files changed, 88 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 7ea3dd87..fa8ed67c 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -103,7 +103,7 @@ class ElasticsearchWriter implements SinkWriter { RestClient.builder(hosts.toArray(new HttpHost[0])), networkClientConfig)); this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); - this.requestIndexer = new DefaultRequestIndexer(); + this.requestIndexer = new DefaultRequestIndexer(metricGroup.getNumRecordsSendCounter()); checkNotNull(metricGroup); metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); @@ -294,9 +294,16 @@ private boolean isClosed() { private class DefaultRequestIndexer implements RequestIndexer { + private final Counter numRecordsSendCounter; + + public DefaultRequestIndexer(Counter numRecordsSendCounter) { + this.numRecordsSendCounter = checkNotNull(numRecordsSendCounter); + } + @Override public void add(DeleteRequest... deleteRequests) { for (final DeleteRequest deleteRequest : deleteRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(deleteRequest); } @@ -305,6 +312,7 @@ public void add(DeleteRequest... deleteRequests) { @Override public void add(IndexRequest... indexRequests) { for (final IndexRequest indexRequest : indexRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(indexRequest); } @@ -313,6 +321,7 @@ public void add(IndexRequest... indexRequests) { @Override public void add(UpdateRequest... updateRequests) { for (final UpdateRequest updateRequest : updateRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(updateRequest); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index d0132a0b..3db12a40 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.metrics.Counter; @@ -25,6 +26,7 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.DockerImageVersions; @@ -38,8 +40,11 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; @@ -58,8 +63,11 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.buildMessage; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; @@ -204,6 +212,30 @@ void testIncrementByteOutMetric() throws Exception { } } + @Test + void testIncrementRecordsSendMetric() throws Exception { + final String index = "test-inc-records-send"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional recordsSend = + metricListener.getCounter(MetricNames.NUM_RECORDS_SEND); + writer.write(Tuple2.of(1, buildMessage(1)), null); + // Update existing index + writer.write(Tuple2.of(1, "u" + buildMessage(2)), null); + // Delete index + writer.write(Tuple2.of(1, "d" + buildMessage(3)), null); + + writer.blockingFlushAllActions(); + + assertTrue(recordsSend.isPresent()); + assertEquals(recordsSend.get().getCount(), 3L); + } + } + @Test void testCurrentSendTime() throws Exception { final String index = "test-current-send-time"; @@ -239,9 +271,9 @@ private ElasticsearchWriter> createWriter( boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig, SinkWriterMetricGroup metricGroup) { - return new ElasticsearchWriter>( + return new ElasticsearchWriter<>( Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), - TestEmitter.jsonEmitter(index, context.getDataFieldName()), + new UpdatingEmitter(index, context.getDataFieldName()), flushOnCheckpoint, bulkProcessorConfig, new TestBulkProcessorBuilderFactory(), @@ -310,6 +342,50 @@ public void accept( } } + private static class UpdatingEmitter implements ElasticsearchEmitter> { + + private final String dataFieldName; + private final String index; + + UpdatingEmitter(String index, String dataFieldName) { + this.index = index; + this.dataFieldName = dataFieldName; + } + + @Override + public void emit( + Tuple2 element, + SinkWriter.Context context, + RequestIndexer indexer) { + + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + + final char action = element.f1.charAt(0); + final String id = element.f0.toString(); + switch (action) { + case 'd': + { + indexer.add(new DeleteRequest(index).id(id)); + break; + } + case 'u': + { + indexer.add(new UpdateRequest().index(index).id(id).doc(document)); + break; + } + default: + { + indexer.add( + new IndexRequest(index) + .id(id) + .type(DOCUMENT_TYPE) + .source(document)); + } + } + } + } + private static class TestClient extends TestClientBase { TestClient(RestHighLevelClient client) { From 52e97351f75049221f51129f1fafa0d4a7e78809 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Thu, 17 Mar 2022 11:51:22 +0100 Subject: [PATCH 199/207] [FLINK-26638][connectors/elasticsearch] Revert Table-API implementation to SinkFunction-based one --- .../table/AbstractTimeIndexGenerator.java | 41 ++ .../table/ElasticsearchConfiguration.java | 169 ++++++++ .../table/ElasticsearchConnectorOptions.java | 171 ++++++++ .../table/ElasticsearchValidationUtils.java | 94 +++++ .../elasticsearch/table/IndexGenerator.java | 39 ++ .../table/IndexGeneratorBase.java | 52 +++ .../table/IndexGeneratorFactory.java | 312 ++++++++++++++ .../elasticsearch/table/KeyExtractor.java | 130 ++++++ .../elasticsearch/table/RequestFactory.java | 54 +++ .../table/RowElasticsearchSinkFunction.java | 140 ++++++ .../table/StaticIndexGenerator.java | 35 ++ .../table/IndexGeneratorFactoryTest.java | 282 +++++++++++++ .../elasticsearch/table/KeyExtractorTest.java | 135 ++++++ .../elasticsearch/table/TestContext.java | 72 ++++ .../table/Elasticsearch6Configuration.java | 79 ++++ .../table/Elasticsearch6DynamicSink.java | 335 +++++++++++++++ .../Elasticsearch6DynamicSinkFactory.java | 186 ++++++++ .../org.apache.flink.table.factories.Factory | 2 +- .../Elasticsearch6DynamicSinkFactoryTest.java | 250 +++++++++++ .../Elasticsearch6DynamicSinkITCase.java | 399 ++++++++++++++++++ .../table/Elasticsearch6DynamicSinkTest.java | 298 +++++++++++++ .../table/Elasticsearch7Configuration.java | 70 +++ .../table/Elasticsearch7DynamicSink.java | 335 +++++++++++++++ .../Elasticsearch7DynamicSinkFactory.java | 186 ++++++++ .../org.apache.flink.table.factories.Factory | 2 +- .../Elasticsearch7DynamicSinkFactoryTest.java | 234 ++++++++++ .../Elasticsearch7DynamicSinkITCase.java | 376 +++++++++++++++++ .../table/Elasticsearch7DynamicSinkTest.java | 298 +++++++++++++ 28 files changed, 4774 insertions(+), 2 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java new file mode 100644 index 00000000..6c22cf3b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java @@ -0,0 +1,41 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.time.format.DateTimeFormatter; + +/** Abstract class for time related {@link IndexGenerator}. */ +@Internal +abstract class AbstractTimeIndexGenerator extends IndexGeneratorBase { + + private final String dateTimeFormat; + protected transient DateTimeFormatter dateTimeFormatter; + + public AbstractTimeIndexGenerator(String index, String dateTimeFormat) { + super(index); + this.dateTimeFormat = dateTimeFormat; + } + + @Override + public void open() { + this.dateTimeFormatter = DateTimeFormatter.ofPattern(dateTimeFormat); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java new file mode 100644 index 00000000..04c76333 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java @@ -0,0 +1,169 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.util.IgnoringFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.util.RetryRejectedExecutionFailureHandler; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.util.InstantiationUtil; + +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** Accessor methods to elasticsearch options. */ +@Internal +class ElasticsearchConfiguration { + protected final ReadableConfig config; + private final ClassLoader classLoader; + + ElasticsearchConfiguration(ReadableConfig config, ClassLoader classLoader) { + this.config = config; + this.classLoader = classLoader; + } + + public ActionRequestFailureHandler getFailureHandler() { + final ActionRequestFailureHandler failureHandler; + String value = config.get(FAILURE_HANDLER_OPTION); + switch (value.toUpperCase()) { + case "FAIL": + failureHandler = new NoOpFailureHandler(); + break; + case "IGNORE": + failureHandler = new IgnoringFailureHandler(); + break; + case "RETRY-REJECTED": + failureHandler = new RetryRejectedExecutionFailureHandler(); + break; + default: + try { + Class failureHandlerClass = Class.forName(value, false, classLoader); + failureHandler = + (ActionRequestFailureHandler) + InstantiationUtil.instantiate(failureHandlerClass); + } catch (ClassNotFoundException e) { + throw new ValidationException( + "Could not instantiate the failure handler class: " + value, e); + } + break; + } + return failureHandler; + } + + public String getDocumentType() { + return config.get(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION); + } + + public int getBulkFlushMaxActions() { + int maxActions = config.get(ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxActions == 0 ? -1 : maxActions; + } + + public long getBulkFlushMaxByteSize() { + long maxSize = + config.get(ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION).getBytes(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return maxSize == 0 ? -1 : maxSize; + } + + public long getBulkFlushInterval() { + long interval = config.get(BULK_FLUSH_INTERVAL_OPTION).toMillis(); + // convert 0 to -1, because Elasticsearch client use -1 to disable this configuration. + return interval == 0 ? -1 : interval; + } + + public Optional getUsername() { + return config.getOptional(USERNAME_OPTION); + } + + public Optional getPassword() { + return config.getOptional(PASSWORD_OPTION); + } + + public boolean isBulkFlushBackoffEnabled() { + return config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION) + != ElasticsearchConnectorOptions.BackOffType.DISABLED; + } + + public Optional getBulkFlushBackoffType() { + switch (config.get(BULK_FLUSH_BACKOFF_TYPE_OPTION)) { + case CONSTANT: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.CONSTANT); + case EXPONENTIAL: + return Optional.of(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + default: + return Optional.empty(); + } + } + + public Optional getBulkFlushBackoffRetries() { + return config.getOptional(BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION); + } + + public Optional getBulkFlushBackoffDelay() { + return config.getOptional(BULK_FLUSH_BACKOFF_DELAY_OPTION).map(Duration::toMillis); + } + + public boolean isDisableFlushOnCheckpoint() { + return !config.get(ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION); + } + + public String getIndex() { + return config.get(ElasticsearchConnectorOptions.INDEX_OPTION); + } + + public String getKeyDelimiter() { + return config.get(ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION); + } + + public Optional getPathPrefix() { + return config.getOptional(ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ElasticsearchConfiguration that = (ElasticsearchConfiguration) o; + return Objects.equals(config, that.config) && Objects.equals(classLoader, that.classLoader); + } + + @Override + public int hashCode() { + return Objects.hash(config, classLoader); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java new file mode 100644 index 00000000..1b0fc5e4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -0,0 +1,171 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; + +/** Options for the Elasticsearch connector. */ +@PublicEvolving +public class ElasticsearchConnectorOptions { + + public static final ConfigOption> HOSTS_OPTION = + ConfigOptions.key("hosts") + .stringType() + .asList() + .noDefaultValue() + .withDescription("Elasticsearch hosts to connect to."); + + public static final ConfigOption INDEX_OPTION = + ConfigOptions.key("index") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch index for every record."); + + public static final ConfigOption DOCUMENT_TYPE_OPTION = + ConfigOptions.key("document-type") + .stringType() + .noDefaultValue() + .withDescription("Elasticsearch document type."); + + public static final ConfigOption PASSWORD_OPTION = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("Password used to connect to Elasticsearch instance."); + + public static final ConfigOption USERNAME_OPTION = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("Username used to connect to Elasticsearch instance."); + + public static final ConfigOption KEY_DELIMITER_OPTION = + ConfigOptions.key("document-id.key-delimiter") + .stringType() + .defaultValue("_") + .withDescription( + "Delimiter for composite keys e.g., \"$\" would result in IDs \"KEY1$KEY2$KEY3\"."); + + public static final ConfigOption FAILURE_HANDLER_OPTION = + ConfigOptions.key("failure-handler") + .stringType() + .defaultValue("fail") + .withDescription( + Description.builder() + .text( + "Failure handling strategy in case a request to Elasticsearch fails") + .list( + text( + "\"fail\" (throws an exception if a request fails and thus causes a job failure)"), + text( + "\"ignore\" (ignores failures and drops the request)"), + text( + "\"retry-rejected\" (re-adds requests that have failed due to queue capacity saturation)"), + text( + "\"class name\" for failure handling with a ActionRequestFailureHandler subclass")) + .build()); + + public static final ConfigOption FLUSH_ON_CHECKPOINT_OPTION = + ConfigOptions.key("sink.flush-on-checkpoint") + .booleanType() + .defaultValue(true) + .withDescription("Disables flushing on checkpoint"); + + public static final ConfigOption BULK_FLUSH_MAX_ACTIONS_OPTION = + ConfigOptions.key("sink.bulk-flush.max-actions") + .intType() + .defaultValue(1000) + .withDescription("Maximum number of actions to buffer for each bulk request."); + + public static final ConfigOption BULK_FLASH_MAX_SIZE_OPTION = + ConfigOptions.key("sink.bulk-flush.max-size") + .memoryType() + .defaultValue(MemorySize.parse("2mb")) + .withDescription("Maximum size of buffered actions per bulk request"); + + public static final ConfigOption BULK_FLUSH_INTERVAL_OPTION = + ConfigOptions.key("sink.bulk-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(1)) + .withDescription("Bulk flush interval"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_TYPE_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.strategy") + .enumType(BackOffType.class) + .defaultValue(BackOffType.DISABLED) + .withDescription("Backoff strategy"); + + public static final ConfigOption BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.max-retries") + .intType() + .noDefaultValue() + .withDescription("Maximum number of retries."); + + public static final ConfigOption BULK_FLUSH_BACKOFF_DELAY_OPTION = + ConfigOptions.key("sink.bulk-flush.backoff.delay") + .durationType() + .noDefaultValue() + .withDescription("Delay between each backoff attempt."); + + public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = + ConfigOptions.key("connection.max-retry-timeout") + .durationType() + .noDefaultValue() + .withDescription("Maximum timeout between retries."); + + public static final ConfigOption CONNECTION_PATH_PREFIX = + ConfigOptions.key("connection.path-prefix") + .stringType() + .noDefaultValue() + .withDescription("Prefix string to be added to every REST communication."); + + public static final ConfigOption FORMAT_OPTION = + ConfigOptions.key("format") + .stringType() + .defaultValue("json") + .withDescription( + "The format must produce a valid JSON document. " + + "Please refer to the documentation on formats for more details."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** + * Backoff strategy. Extends {@link ElasticsearchSinkBase.FlushBackoffType} with {@code + * DISABLED} option. + */ + public enum BackOffType { + DISABLED, + CONSTANT, + EXPONENTIAL + } + + private ElasticsearchConnectorOptions() {} +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java new file mode 100644 index 00000000..6452d006 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java @@ -0,0 +1,94 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** Utility methods for validating Elasticsearch properties. */ +@Internal +class ElasticsearchValidationUtils { + + private static final Set ILLEGAL_PRIMARY_KEY_TYPES = new LinkedHashSet<>(); + + static { + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ARRAY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MAP); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.MULTISET); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.STRUCTURED_TYPE); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.ROW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.RAW); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BINARY); + ILLEGAL_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARBINARY); + } + + /** + * Checks that the table does not have primary key defined on illegal types. In Elasticsearch + * the primary key is used to calculate the Elasticsearch document id, which is a string of up + * to 512 bytes. It cannot have whitespaces. As of now it is calculated by concatenating the + * fields. Certain types do not have a good string representation to be used in this scenario. + * The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link + * LogicalTypeRoot#RAW} type. + */ + public static void validatePrimaryKey(TableSchema schema) { + schema.getPrimaryKey() + .ifPresent( + key -> { + List illegalTypes = + key.getColumns().stream() + .map( + fieldName -> { + LogicalType logicalType = + schema.getFieldDataType(fieldName) + .get() + .getLogicalType(); + if (logicalType.is( + LogicalTypeRoot.DISTINCT_TYPE)) { + return ((DistinctType) logicalType) + .getSourceType() + .getTypeRoot(); + } else { + return logicalType.getTypeRoot(); + } + }) + .filter(ILLEGAL_PRIMARY_KEY_TYPES::contains) + .collect(Collectors.toList()); + + if (!illegalTypes.isEmpty()) { + throw new ValidationException( + String.format( + "The table has a primary key on columns of illegal types: %s.\n" + + " Elasticsearch sink does not support primary keys on columns of types: %s.", + illegalTypes, ILLEGAL_PRIMARY_KEY_TYPES)); + } + }); + } + + private ElasticsearchValidationUtils() {} +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java new file mode 100644 index 00000000..636f3409 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java @@ -0,0 +1,39 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; + +import java.io.Serializable; + +/** This interface is responsible to generate index name from given {@link Row} record. */ +@Internal +interface IndexGenerator extends Serializable { + + /** + * Initialize the index generator, this will be called only once before {@link + * #generate(RowData)} is called. + */ + default void open() {} + + /** Generate index name according the the given row. */ + String generate(RowData row); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java new file mode 100644 index 00000000..adfcaa46 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java @@ -0,0 +1,52 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; + +import java.util.Objects; + +/** Base class for {@link IndexGenerator}. */ +@Internal +public abstract class IndexGeneratorBase implements IndexGenerator { + + private static final long serialVersionUID = 1L; + protected final String index; + + public IndexGeneratorBase(String index) { + this.index = index; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IndexGeneratorBase)) { + return false; + } + IndexGeneratorBase that = (IndexGeneratorBase) o; + return index.equals(that.index); + } + + @Override + public int hashCode() { + return Objects.hash(index); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java new file mode 100644 index 00000000..48f0107b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -0,0 +1,312 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Factory of {@link IndexGenerator}. + * + *

Flink supports both static index and dynamic index. + * + *

If you want to have a static index, this option value should be a plain string, e.g. + * 'myusers', all the records will be consistently written into "myusers" index. + * + *

If you want to have a dynamic index, you can use '{field_name}' to reference a field value in + * the record to dynamically generate a target index. You can also use + * '{field_name|date_format_string}' to convert a field value of TIMESTAMP/DATE/TIME type into the + * format specified by date_format_string. The date_format_string is compatible with {@link + * java.text.SimpleDateFormat}. For example, if the option value is 'myusers_{log_ts|yyyy-MM-dd}', + * then a record with log_ts field value 2020-03-27 12:25:55 will be written into + * "myusers_2020-03-27" index. + */ +@Internal +final class IndexGeneratorFactory { + + private IndexGeneratorFactory() {} + + public static IndexGenerator createIndexGenerator(String index, TableSchema schema) { + return createIndexGenerator(index, schema, ZoneId.systemDefault()); + } + + public static IndexGenerator createIndexGenerator( + String index, TableSchema schema, ZoneId localTimeZoneId) { + final IndexHelper indexHelper = new IndexHelper(); + if (indexHelper.checkIsDynamicIndex(index)) { + return createRuntimeIndexGenerator( + index, + schema.getFieldNames(), + schema.getFieldDataTypes(), + indexHelper, + localTimeZoneId); + } else { + return new StaticIndexGenerator(index); + } + } + + interface DynamicFormatter extends Serializable { + String format(@Nonnull Object fieldValue, DateTimeFormatter formatter); + } + + private static IndexGenerator createRuntimeIndexGenerator( + String index, + String[] fieldNames, + DataType[] fieldTypes, + IndexHelper indexHelper, + ZoneId localTimeZoneId) { + final String dynamicIndexPatternStr = indexHelper.extractDynamicIndexPatternStr(index); + final String indexPrefix = index.substring(0, index.indexOf(dynamicIndexPatternStr)); + final String indexSuffix = + index.substring(indexPrefix.length() + dynamicIndexPatternStr.length()); + + if (indexHelper.checkIsDynamicIndexWithSystemTimeFormat(index)) { + final String dateTimeFormat = + indexHelper.extractDateFormat( + index, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + return indexPrefix + .concat(LocalDateTime.now(localTimeZoneId).format(dateTimeFormatter)) + .concat(indexSuffix); + } + }; + } + + final boolean isDynamicIndexWithFormat = indexHelper.checkIsDynamicIndexWithFormat(index); + final int indexFieldPos = + indexHelper.extractIndexFieldPos(index, fieldNames, isDynamicIndexWithFormat); + final LogicalType indexFieldType = fieldTypes[indexFieldPos].getLogicalType(); + final LogicalTypeRoot indexFieldLogicalTypeRoot = indexFieldType.getTypeRoot(); + + // validate index field type + indexHelper.validateIndexFieldType(indexFieldLogicalTypeRoot); + + // time extract dynamic index pattern + final RowData.FieldGetter fieldGetter = + RowData.createFieldGetter(indexFieldType, indexFieldPos); + + if (isDynamicIndexWithFormat) { + final String dateTimeFormat = + indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); + DynamicFormatter formatFunction = + createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + + return new AbstractTimeIndexGenerator(index, dateTimeFormat) { + @Override + public String generate(RowData row) { + Object fieldOrNull = fieldGetter.getFieldOrNull(row); + final String formattedField; + // TODO we can possibly optimize it to use the nullability of the field + if (fieldOrNull != null) { + formattedField = formatFunction.format(fieldOrNull, dateTimeFormatter); + } else { + formattedField = "null"; + } + return indexPrefix.concat(formattedField).concat(indexSuffix); + } + }; + } + // general dynamic index pattern + return new IndexGeneratorBase(index) { + @Override + public String generate(RowData row) { + Object indexField = fieldGetter.getFieldOrNull(row); + return indexPrefix + .concat(indexField == null ? "null" : indexField.toString()) + .concat(indexSuffix); + } + }; + } + + private static DynamicFormatter createFormatFunction( + LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + switch (indexFieldLogicalTypeRoot) { + case DATE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalDate.ofEpochDay(indexField).format(dateTimeFormatter); + }; + case TIME_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + Integer indexField = (Integer) value; + return LocalTime.ofNanoOfDay(indexField * 1_000_000L).format(dateTimeFormatter); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toLocalDateTime().format(dateTimeFormatter); + }; + case TIMESTAMP_WITH_TIME_ZONE: + throw new UnsupportedOperationException( + "TIMESTAMP_WITH_TIME_ZONE is not supported yet"); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return (value, dateTimeFormatter) -> { + TimestampData indexField = (TimestampData) value; + return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + }; + default: + throw new TableException( + String.format( + "Unsupported type '%s' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP.", + indexFieldType)); + } + } + + /** + * Helper class for {@link IndexGeneratorFactory}, this helper can use to validate index field + * type ans parse index format from pattern. + */ + static class IndexHelper { + private static final Pattern dynamicIndexPattern = Pattern.compile("\\{[^\\{\\}]+\\}?"); + private static final Pattern dynamicIndexTimeExtractPattern = + Pattern.compile(".*\\{.+\\|.*\\}.*"); + private static final Pattern dynamicIndexSystemTimeExtractPattern = + Pattern.compile( + ".*\\{\\s*(now\\(\\s*\\)|NOW\\(\\s*\\)|current_timestamp|CURRENT_TIMESTAMP)\\s*\\|.*\\}.*"); + private static final List supportedTypes = new ArrayList<>(); + private static final Map defaultFormats = new HashMap<>(); + + static { + // time related types + supportedTypes.add(LogicalTypeRoot.DATE); + supportedTypes.add(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE); + supportedTypes.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + // general types + supportedTypes.add(LogicalTypeRoot.VARCHAR); + supportedTypes.add(LogicalTypeRoot.CHAR); + supportedTypes.add(LogicalTypeRoot.TINYINT); + supportedTypes.add(LogicalTypeRoot.INTEGER); + supportedTypes.add(LogicalTypeRoot.BIGINT); + } + + static { + defaultFormats.put(LogicalTypeRoot.DATE, "yyyy_MM_dd"); + defaultFormats.put(LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE, "HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put(LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, "yyyy_MM_dd_HH_mm_ss"); + defaultFormats.put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, "yyyy_MM_dd_HH_mm_ssX"); + } + + /** Validate the index field Type. */ + void validateIndexFieldType(LogicalTypeRoot logicalType) { + if (!supportedTypes.contains(logicalType)) { + throw new IllegalArgumentException( + String.format( + "Unsupported type %s of index field, " + "Supported types are: %s", + logicalType, supportedTypes)); + } + } + + /** Get the default date format. */ + String getDefaultFormat(LogicalTypeRoot logicalType) { + return defaultFormats.get(logicalType); + } + + /** Check general dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndex(String index) { + final Matcher matcher = dynamicIndexPattern.matcher(index); + int count = 0; + while (matcher.find()) { + count++; + } + if (count > 1) { + throw new TableException( + String.format( + "Chaining dynamic index pattern %s is not supported," + + " only support single dynamic index pattern.", + index)); + } + return count == 1; + } + + /** Check time extract dynamic index is enabled or not by index pattern. */ + boolean checkIsDynamicIndexWithFormat(String index) { + return dynamicIndexTimeExtractPattern.matcher(index).matches(); + } + + /** Check generate dynamic index is from system time or not. */ + boolean checkIsDynamicIndexWithSystemTimeFormat(String index) { + return dynamicIndexSystemTimeExtractPattern.matcher(index).matches(); + } + + /** Extract dynamic index pattern string from index pattern string. */ + String extractDynamicIndexPatternStr(String index) { + int start = index.indexOf("{"); + int end = index.lastIndexOf("}"); + return index.substring(start, end + 1); + } + + /** Extract index field position in a fieldNames, return the field position. */ + int extractIndexFieldPos( + String index, String[] fieldNames, boolean isDynamicIndexWithFormat) { + List fieldList = Arrays.asList(fieldNames); + String indexFieldName; + if (isDynamicIndexWithFormat) { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("|")); + } else { + indexFieldName = index.substring(index.indexOf("{") + 1, index.indexOf("}")); + } + if (!fieldList.contains(indexFieldName)) { + throw new TableException( + String.format( + "Unknown field '%s' in index pattern '%s', please check the field name.", + indexFieldName, index)); + } + return fieldList.indexOf(indexFieldName); + } + + /** Extract dateTime format by the date format that extracted from index pattern string. */ + private String extractDateFormat(String index, LogicalTypeRoot logicalType) { + String format = index.substring(index.indexOf("|") + 1, index.indexOf("}")); + if ("".equals(format)) { + format = getDefaultFormat(logicalType); + } + return format; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java new file mode 100644 index 00000000..ae7c522b --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java @@ -0,0 +1,130 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; + +import java.io.Serializable; +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalTime; +import java.time.Period; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +/** An extractor for a Elasticsearch key from a {@link RowData}. */ +@Internal +class KeyExtractor implements Function, Serializable { + private final FieldFormatter[] fieldFormatters; + private final String keyDelimiter; + + private interface FieldFormatter extends Serializable { + String format(RowData rowData); + } + + private KeyExtractor(FieldFormatter[] fieldFormatters, String keyDelimiter) { + this.fieldFormatters = fieldFormatters; + this.keyDelimiter = keyDelimiter; + } + + @Override + public String apply(RowData rowData) { + final StringBuilder builder = new StringBuilder(); + for (int i = 0; i < fieldFormatters.length; i++) { + if (i > 0) { + builder.append(keyDelimiter); + } + final String value = fieldFormatters[i].format(rowData); + builder.append(value); + } + return builder.toString(); + } + + private static class ColumnWithIndex { + public TableColumn column; + public int index; + + public ColumnWithIndex(TableColumn column, int index) { + this.column = column; + this.index = index; + } + + public LogicalType getType() { + return column.getType().getLogicalType(); + } + + public int getIndex() { + return index; + } + } + + public static Function createKeyExtractor( + TableSchema schema, String keyDelimiter) { + return schema.getPrimaryKey() + .map( + key -> { + Map namesToColumns = new HashMap<>(); + List tableColumns = schema.getTableColumns(); + for (int i = 0; i < schema.getFieldCount(); i++) { + TableColumn column = tableColumns.get(i); + namesToColumns.put( + column.getName(), new ColumnWithIndex(column, i)); + } + + FieldFormatter[] fieldFormatters = + key.getColumns().stream() + .map(namesToColumns::get) + .map( + column -> + toFormatter( + column.index, column.getType())) + .toArray(FieldFormatter[]::new); + + return (Function) + new KeyExtractor(fieldFormatters, keyDelimiter); + }) + .orElseGet(() -> (Function & Serializable) (row) -> null); + } + + private static FieldFormatter toFormatter(int index, LogicalType type) { + switch (type.getTypeRoot()) { + case DATE: + return (row) -> LocalDate.ofEpochDay(row.getInt(index)).toString(); + case TIME_WITHOUT_TIME_ZONE: + return (row) -> + LocalTime.ofNanoOfDay((long) row.getInt(index) * 1_000_000L).toString(); + case INTERVAL_YEAR_MONTH: + return (row) -> Period.ofDays(row.getInt(index)).toString(); + case INTERVAL_DAY_TIME: + return (row) -> Duration.ofMillis(row.getLong(index)).toString(); + case DISTINCT_TYPE: + return toFormatter(index, ((DistinctType) type).getSourceType()); + default: + RowData.FieldGetter fieldGetter = RowData.createFieldGetter(type, index); + return (row) -> fieldGetter.getFieldOrNull(row).toString(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java new file mode 100644 index 00000000..f5b24180 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java @@ -0,0 +1,54 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import java.io.Serializable; + +/** For version-agnostic creating of {@link ActionRequest}s. */ +@Internal +interface RequestFactory extends Serializable { + /** + * Creates an update request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + UpdateRequest createUpdateRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates an index request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + IndexRequest createIndexRequest( + String index, String docType, String key, XContentType contentType, byte[] document); + + /** + * Creates a delete request to be added to a {@link RequestIndexer}. Note: the type field has + * been deprecated since Elasticsearch 7.x and it would not take any effort. + */ + DeleteRequest createDeleteRequest(String index, String docType, String key); +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java new file mode 100644 index 00000000..7fb1e4d4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java @@ -0,0 +1,140 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.util.Objects; +import java.util.function.Function; + +/** Sink function for converting upserts into Elasticsearch {@link ActionRequest}s. */ +@Internal +class RowElasticsearchSinkFunction implements ElasticsearchSinkFunction { + + private static final long serialVersionUID = 1L; + + private final IndexGenerator indexGenerator; + private final String docType; + private final SerializationSchema serializationSchema; + private final XContentType contentType; + private final RequestFactory requestFactory; + private final Function createKey; + + public RowElasticsearchSinkFunction( + IndexGenerator indexGenerator, + @Nullable String docType, // this is deprecated in es 7+ + SerializationSchema serializationSchema, + XContentType contentType, + RequestFactory requestFactory, + Function createKey) { + this.indexGenerator = Preconditions.checkNotNull(indexGenerator); + this.docType = docType; + this.serializationSchema = Preconditions.checkNotNull(serializationSchema); + this.contentType = Preconditions.checkNotNull(contentType); + this.requestFactory = Preconditions.checkNotNull(requestFactory); + this.createKey = Preconditions.checkNotNull(createKey); + } + + @Override + public void open() { + indexGenerator.open(); + } + + @Override + public void process(RowData element, RuntimeContext ctx, RequestIndexer indexer) { + switch (element.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + processUpsert(element, indexer); + break; + case UPDATE_BEFORE: + case DELETE: + processDelete(element, indexer); + break; + default: + throw new TableException("Unsupported message kind: " + element.getRowKind()); + } + } + + private void processUpsert(RowData row, RequestIndexer indexer) { + final byte[] document = serializationSchema.serialize(row); + final String key = createKey.apply(row); + if (key != null) { + final UpdateRequest updateRequest = + requestFactory.createUpdateRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(updateRequest); + } else { + final IndexRequest indexRequest = + requestFactory.createIndexRequest( + indexGenerator.generate(row), docType, key, contentType, document); + indexer.add(indexRequest); + } + } + + private void processDelete(RowData row, RequestIndexer indexer) { + final String key = createKey.apply(row); + final DeleteRequest deleteRequest = + requestFactory.createDeleteRequest(indexGenerator.generate(row), docType, key); + indexer.add(deleteRequest); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RowElasticsearchSinkFunction that = (RowElasticsearchSinkFunction) o; + return Objects.equals(indexGenerator, that.indexGenerator) + && Objects.equals(docType, that.docType) + && Objects.equals(serializationSchema, that.serializationSchema) + && contentType == that.contentType + && Objects.equals(requestFactory, that.requestFactory) + && Objects.equals(createKey, that.createKey); + } + + @Override + public int hashCode() { + return Objects.hash( + indexGenerator, + docType, + serializationSchema, + contentType, + requestFactory, + createKey); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java new file mode 100644 index 00000000..1ffcac48 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java @@ -0,0 +1,35 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; + +/** A static {@link IndexGenerator} which generate fixed index name. */ +@Internal +final class StaticIndexGenerator extends IndexGeneratorBase { + + public StaticIndexGenerator(String index) { + super(index); + } + + public String generate(RowData row) { + return index; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java new file mode 100644 index 00000000..a5f77595 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java @@ -0,0 +1,282 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.util.TestLogger; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import java.time.temporal.UnsupportedTemporalTypeException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Tests for {@link IndexGeneratorFactory}. */ +public class IndexGeneratorFactoryTest extends TestLogger { + + private TableSchema schema; + private List rows; + + @Before + public void prepareData() { + schema = + new TableSchema.Builder() + .field("id", DataTypes.INT()) + .field("item", DataTypes.STRING()) + .field("log_ts", DataTypes.BIGINT()) + .field("log_date", DataTypes.DATE()) + .field("log_time", DataTypes.TIME()) + .field("order_timestamp", DataTypes.TIMESTAMP()) + .field("local_timestamp", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("status", DataTypes.BOOLEAN()) + .build(); + + rows = new ArrayList<>(); + rows.add( + GenericRowData.of( + 1, + StringData.fromString("apple"), + Timestamp.valueOf("2020-03-18 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-18").toEpochDay(), + (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), + TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), + true)); + rows.add( + GenericRowData.of( + 2, + StringData.fromString("peanut"), + Timestamp.valueOf("2020-03-19 12:12:14").getTime(), + (int) LocalDate.parse("2020-03-19").toEpochDay(), + (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), + TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), + TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), + false)); + } + + @Test + public void testDynamicIndexFromTimestamp() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH-ss}_index", schema); + indexGenerator.open(); + Assert.assertEquals("2020_03_18_12-14_index", indexGenerator.generate(rows.get(0))); + IndexGenerator indexGenerator1 = + IndexGeneratorFactory.createIndexGenerator( + "{order_timestamp|yyyy_MM_dd_HH_mm}_index", schema); + indexGenerator1.open(); + Assert.assertEquals("2020_03_19_12_22_index", indexGenerator1.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromDate() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020/03/18", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020/03/19", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromTime() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|HH-mm}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12-12", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12-22", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormat() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{log_time|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-12_12_14", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-12_22_21", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexFromSystemTime() { + List supportedUseCases = + Arrays.asList( + "now()", + "NOW()", + "now( )", + "NOW(\t)", + "\t NOW( ) \t", + "current_timestamp", + "CURRENT_TIMESTAMP", + "\tcurrent_timestamp\t", + " current_timestamp "); + + supportedUseCases.stream() + .forEach( + f -> { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), schema); + indexGenerator.open(); + // The date may change during the running of the unit test. + // Generate expected index-name based on the current time + // before and after calling the generate method. + String expectedIndex1 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + String actualIndex = indexGenerator.generate(rows.get(1)); + String expectedIndex2 = + "my-index-" + LocalDateTime.now().format(dateTimeFormatter); + Assert.assertTrue( + actualIndex.equals(expectedIndex1) + || actualIndex.equals(expectedIndex2)); + }); + + List invalidUseCases = + Arrays.asList( + "now", + "now(", + "NOW", + "NOW)", + "current_timestamp()", + "CURRENT_TIMESTAMP()", + "CURRENT_timestamp"); + invalidUseCases.stream() + .forEach( + f -> { + String expectedExceptionMsg = + String.format( + "Unknown field '%s' in index pattern 'my-index-{%s|yyyy_MM_dd}'," + + " please check the field name.", + f, f); + try { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + String.format("my-index-{%s|yyyy_MM_dd}", f), + schema); + indexGenerator.open(); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + }); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); + indexGenerator.open(); + Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testGeneralDynamicIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("index_{item}", schema); + indexGenerator.open(); + Assert.assertEquals("index_apple", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("index_peanut", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testStaticIndex() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator("my-index", schema); + indexGenerator.open(); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testUnknownField() { + String expectedExceptionMsg = + "Unknown field 'unknown_ts' in index pattern 'my-index-{unknown_ts|yyyy-MM-dd}'," + + " please check the field name."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{unknown_ts|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(e.getMessage(), expectedExceptionMsg); + } + } + + @Test + public void testUnsupportedTimeType() { + String expectedExceptionMsg = + "Unsupported type 'INT' found in Elasticsearch dynamic index field, " + + "time-related pattern only support types are: DATE,TIME,TIMESTAMP."; + try { + IndexGeneratorFactory.createIndexGenerator("my-index-{id|yyyy-MM-dd}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedMultiParametersType() { + String expectedExceptionMsg = + "Chaining dynamic index pattern my-index-{local_date}-{local_time} is not supported," + + " only support single dynamic index pattern."; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_date}-{local_time}", schema); + } catch (TableException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testDynamicIndexUnsupportedFormat() { + String expectedExceptionMsg = "Unsupported field: HourOfDay"; + try { + IndexGeneratorFactory.createIndexGenerator( + "my-index-{log_date|yyyy/MM/dd HH:mm}", schema); + } catch (UnsupportedTemporalTypeException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } + + @Test + public void testUnsupportedIndexFieldType() { + String expectedExceptionMsg = + "Unsupported type BOOLEAN of index field, Supported types are:" + + " [DATE, TIME_WITHOUT_TIME_ZONE, TIMESTAMP_WITHOUT_TIME_ZONE, TIMESTAMP_WITH_TIME_ZONE," + + " TIMESTAMP_WITH_LOCAL_TIME_ZONE, VARCHAR, CHAR, TINYINT, INTEGER, BIGINT]"; + try { + IndexGeneratorFactory.createIndexGenerator("index_{status}", schema); + } catch (IllegalArgumentException e) { + Assert.assertEquals(expectedExceptionMsg, e.getMessage()); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java new file mode 100644 index 00000000..bcfb68da --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.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 org.apache.flink.streaming.connectors.elasticsearch.table; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; + +import org.junit.Test; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.function.Function; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.junit.Assert.assertThat; + +/** Tests for {@link KeyExtractor}. */ +public class KeyExtractorTest { + @Test + public void testSimpleKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .primaryKey("a") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, equalTo("12")); + } + + @Test + public void testNoPrimaryKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD"))); + assertThat(key, nullValue()); + } + + @Test + public void testTwoFieldsKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.BIGINT().notNull()) + .field("b", DataTypes.STRING()) + .field("c", DataTypes.TIMESTAMP().notNull()) + .primaryKey("a", "c") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + 12L, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")))); + assertThat(key, equalTo("12_2012-12-12T12:12:12")); + } + + @Test + public void testAllTypesKey() { + TableSchema schema = + TableSchema.builder() + .field("a", DataTypes.TINYINT().notNull()) + .field("b", DataTypes.SMALLINT().notNull()) + .field("c", DataTypes.INT().notNull()) + .field("d", DataTypes.BIGINT().notNull()) + .field("e", DataTypes.BOOLEAN().notNull()) + .field("f", DataTypes.FLOAT().notNull()) + .field("g", DataTypes.DOUBLE().notNull()) + .field("h", DataTypes.STRING().notNull()) + .field("i", DataTypes.TIMESTAMP().notNull()) + .field("j", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE().notNull()) + .field("k", DataTypes.TIME().notNull()) + .field("l", DataTypes.DATE().notNull()) + .primaryKey("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l") + .build(); + + Function keyExtractor = KeyExtractor.createKeyExtractor(schema, "_"); + + String key = + keyExtractor.apply( + GenericRowData.of( + (byte) 1, + (short) 2, + 3, + (long) 4, + true, + 1.0f, + 2.0d, + StringData.fromString("ABCD"), + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12")), + TimestampData.fromInstant(Instant.parse("2013-01-13T13:13:13Z")), + (int) (LocalTime.parse("14:14:14").toNanoOfDay() / 1_000_000), + (int) LocalDate.parse("2015-05-15").toEpochDay())); + assertThat( + key, + equalTo( + "1_2_3_4_true_1.0_2.0_ABCD_2012-12-12T12:12:12_2013-01-13T13:13:13_14:14:14_2015-05-15")); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java new file mode 100644 index 00000000..40405142 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java @@ -0,0 +1,72 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for mocking {@link DynamicTableFactory.Context}. */ +class TestContext { + + private ResolvedSchema schema = ResolvedSchema.of(Column.physical("a", DataTypes.TIME())); + + private final Map options = new HashMap<>(); + + public static TestContext context() { + return new TestContext(); + } + + public TestContext withSchema(ResolvedSchema schema) { + this.schema = schema; + return this; + } + + DynamicTableFactory.Context build() { + return new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of("default", "default", "t1"), + new ResolvedCatalogTable( + CatalogTable.of( + Schema.newBuilder().fromResolvedSchema(schema).build(), + "mock context", + Collections.emptyList(), + options), + schema), + Collections.emptyMap(), + new Configuration(), + TestContext.class.getClassLoader(), + false); + } + + public TestContext withOption(String key, String value) { + options.put(key, value); + return this; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java new file mode 100644 index 00000000..8b833216 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java @@ -0,0 +1,79 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 6 specific configuration. */ +@Internal +final class Elasticsearch6Configuration extends ElasticsearchConfiguration { + Elasticsearch6Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch6Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + /** + * Parse Hosts String to list. + * + *

Hosts String format was given as following: + * + *

+     *     connector.hosts = http://host_name:9092;http://host_name:9093
+     * 
+ */ + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java new file mode 100644 index 00000000..0c7c756d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -0,0 +1,335 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch6.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@PublicEvolving +final class Elasticsearch6DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch6RequestFactory REQUEST_FACTORY = new Elasticsearch6RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch6Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch6DynamicSink( + EncodingFormat> format, + Elasticsearch6Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + config.getDocumentType(), + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch6"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, username, password); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch6RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, docType, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index, docType, key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, docType, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch6DynamicSink that = (Elasticsearch6DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java new file mode 100644 index 00000000..ff600a7a --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -0,0 +1,186 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch6DynamicSink}. */ +@Internal +public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION, DOCUMENT_TYPE_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch6Configuration config = + new Elasticsearch6Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch6DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch6Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-6"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index bb5a894b..29a85938 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.connector.elasticsearch.table.Elasticsearch6DynamicSinkFactory +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch6DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java new file mode 100644 index 00000000..e99abbe9 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java @@ -0,0 +1,250 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch6DynamicSinkFactory}. */ +public class Elasticsearch6DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "document-type\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("document-type", "MyType") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "MyIndex") + .withOption("document-type", "MyType") + .withOption("hosts", "wrong-host") + .build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), "MyType") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java new file mode 100644 index 00000000..0ebc52b5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -0,0 +1,399 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch6DynamicSink}. */ +public class Elasticsearch6DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + String myType = "MyType"; + Elasticsearch6DynamicSinkFactory sinkFactory = new Elasticsearch6DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions + .DOCUMENT_TYPE_OPTION + .key(), + myType) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL,\n" + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits() < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits() < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits() < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + String myType = "MyType"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-6") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), myType) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")) + .actionGet() + .getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java new file mode 100644 index 00000000..f8ab0ab0 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch6DynamicSink} parameters. */ +public class Elasticsearch6DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch6DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch6DynamicSink testSink = + new Elasticsearch6DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch6Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch6DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch6DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java new file mode 100644 index 00000000..6bd28cf4 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java @@ -0,0 +1,70 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; + +import org.apache.http.HttpHost; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; + +/** Elasticsearch 7 specific configuration. */ +@Internal +final class Elasticsearch7Configuration extends ElasticsearchConfiguration { + Elasticsearch7Configuration(ReadableConfig config, ClassLoader classLoader) { + super(config, classLoader); + } + + public List getHosts() { + return config.get(HOSTS_OPTION).stream() + .map(Elasticsearch7Configuration::validateAndParseHostsString) + .collect(Collectors.toList()); + } + + private static HttpHost validateAndParseHostsString(String host) { + try { + HttpHost httpHost = HttpHost.create(host); + if (httpHost.getPort() < 0) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing port.", + host, HOSTS_OPTION.key())); + } + + if (httpHost.getSchemeName() == null) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'. Missing scheme.", + host, HOSTS_OPTION.key())); + } + return httpHost; + } catch (Exception e) { + throw new ValidationException( + String.format( + "Could not parse host '%s' in option '%s'. It should follow the format 'http://host_name:port'.", + host, HOSTS_OPTION.key()), + e); + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java new file mode 100644 index 00000000..1926e445 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java @@ -0,0 +1,335 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.streaming.connectors.elasticsearch7.RestClientFactory; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; + +import org.apache.http.HttpHost; +import org.apache.http.auth.AuthScope; +import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.CredentialsProvider; +import org.apache.http.impl.client.BasicCredentialsProvider; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; +import org.elasticsearch.client.RestClientBuilder; +import org.elasticsearch.common.xcontent.XContentType; + +import javax.annotation.Nullable; + +import java.time.ZoneId; +import java.util.List; +import java.util.Objects; + +/** + * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a + * logical description. + */ +@Internal +final class Elasticsearch7DynamicSink implements DynamicTableSink { + @VisibleForTesting + static final Elasticsearch7RequestFactory REQUEST_FACTORY = new Elasticsearch7RequestFactory(); + + private final EncodingFormat> format; + private final TableSchema schema; + private final Elasticsearch7Configuration config; + private final ZoneId localTimeZoneId; + private final boolean isDynamicIndexWithSystemTime; + + public Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId) { + this(format, config, schema, localTimeZoneId, (ElasticsearchSink.Builder::new)); + } + + // -------------------------------------------------------------- + // Hack to make configuration testing possible. + // + // The code in this block should never be used outside of tests. + // Having a way to inject a builder we can assert the builder in + // the test. We can not assert everything though, e.g. it is not + // possible to assert flushing on checkpoint, as it is configured + // on the sink itself. + // -------------------------------------------------------------- + + private final ElasticSearchBuilderProvider builderProvider; + + @FunctionalInterface + interface ElasticSearchBuilderProvider { + ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction); + } + + Elasticsearch7DynamicSink( + EncodingFormat> format, + Elasticsearch7Configuration config, + TableSchema schema, + ZoneId localTimeZoneId, + ElasticSearchBuilderProvider builderProvider) { + this.format = format; + this.schema = schema; + this.config = config; + this.localTimeZoneId = localTimeZoneId; + this.isDynamicIndexWithSystemTime = isDynamicIndexWithSystemTime(); + this.builderProvider = builderProvider; + } + + // -------------------------------------------------------------- + // End of hack to make configuration testing possible + // -------------------------------------------------------------- + + public boolean isDynamicIndexWithSystemTime() { + IndexGeneratorFactory.IndexHelper indexHelper = new IndexGeneratorFactory.IndexHelper(); + return indexHelper.checkIsDynamicIndexWithSystemTimeFormat(config.getIndex()); + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + if (kind != RowKind.UPDATE_BEFORE) { + builder.addContainedKind(kind); + } + } + if (isDynamicIndexWithSystemTime && !requestedMode.containsOnly(RowKind.INSERT)) { + throw new ValidationException( + "Dynamic indexing based on system time only works on append only stream."); + } + return builder.build(); + } + + @Override + public SinkFunctionProvider getSinkRuntimeProvider(Context context) { + return () -> { + SerializationSchema format = + this.format.createRuntimeEncoder(context, schema.toRowDataType()); + + final RowElasticsearchSinkFunction upsertFunction = + new RowElasticsearchSinkFunction( + IndexGeneratorFactory.createIndexGenerator( + config.getIndex(), schema, localTimeZoneId), + null, // this is deprecated in es 7+ + format, + XContentType.JSON, + REQUEST_FACTORY, + KeyExtractor.createKeyExtractor(schema, config.getKeyDelimiter())); + + final ElasticsearchSink.Builder builder = + builderProvider.createBuilder(config.getHosts(), upsertFunction); + + builder.setFailureHandler(config.getFailureHandler()); + builder.setBulkFlushMaxActions(config.getBulkFlushMaxActions()); + builder.setBulkFlushMaxSizeMb((int) (config.getBulkFlushMaxByteSize() >> 20)); + builder.setBulkFlushInterval(config.getBulkFlushInterval()); + builder.setBulkFlushBackoff(config.isBulkFlushBackoffEnabled()); + config.getBulkFlushBackoffType().ifPresent(builder::setBulkFlushBackoffType); + config.getBulkFlushBackoffRetries().ifPresent(builder::setBulkFlushBackoffRetries); + config.getBulkFlushBackoffDelay().ifPresent(builder::setBulkFlushBackoffDelay); + + // we must overwrite the default factory which is defined with a lambda because of a bug + // in shading lambda serialization shading see FLINK-18006 + if (config.getUsername().isPresent() + && config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get()) + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get())) { + builder.setRestClientFactory( + new AuthRestClientFactory( + config.getPathPrefix().orElse(null), + config.getUsername().get(), + config.getPassword().get())); + } else { + builder.setRestClientFactory( + new DefaultRestClientFactory(config.getPathPrefix().orElse(null))); + } + + final ElasticsearchSink sink = builder.build(); + + if (config.isDisableFlushOnCheckpoint()) { + sink.disableFlushOnCheckpoint(); + } + + return sink; + }; + } + + @Override + public DynamicTableSink copy() { + return this; + } + + @Override + public String asSummaryString() { + return "Elasticsearch7"; + } + + /** Serializable {@link RestClientFactory} used by the sink. */ + @VisibleForTesting + static class DefaultRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + + public DefaultRestClientFactory(@Nullable String pathPrefix) { + this.pathPrefix = pathPrefix; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DefaultRestClientFactory that = (DefaultRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix); + } + } + + /** Serializable {@link RestClientFactory} used by the sink which enable authentication. */ + @VisibleForTesting + static class AuthRestClientFactory implements RestClientFactory { + + private final String pathPrefix; + private final String username; + private final String password; + private transient CredentialsProvider credentialsProvider; + + public AuthRestClientFactory( + @Nullable String pathPrefix, String username, String password) { + this.pathPrefix = pathPrefix; + this.password = password; + this.username = username; + } + + @Override + public void configureRestClientBuilder(RestClientBuilder restClientBuilder) { + if (pathPrefix != null) { + restClientBuilder.setPathPrefix(pathPrefix); + } + if (credentialsProvider == null) { + credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + AuthScope.ANY, new UsernamePasswordCredentials(username, password)); + } + restClientBuilder.setHttpClientConfigCallback( + httpAsyncClientBuilder -> + httpAsyncClientBuilder.setDefaultCredentialsProvider( + credentialsProvider)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AuthRestClientFactory that = (AuthRestClientFactory) o; + return Objects.equals(pathPrefix, that.pathPrefix) + && Objects.equals(username, that.username) + && Objects.equals(password, that.password); + } + + @Override + public int hashCode() { + return Objects.hash(pathPrefix, password, username); + } + } + + /** + * Version-specific creation of {@link org.elasticsearch.action.ActionRequest}s used by the + * sink. + */ + private static class Elasticsearch7RequestFactory implements RequestFactory { + @Override + public UpdateRequest createUpdateRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new UpdateRequest(index, key) + .doc(document, contentType) + .upsert(document, contentType); + } + + @Override + public IndexRequest createIndexRequest( + String index, + String docType, + String key, + XContentType contentType, + byte[] document) { + return new IndexRequest(index).id(key).source(document, contentType); + } + + @Override + public DeleteRequest createDeleteRequest(String index, String docType, String key) { + return new DeleteRequest(index, key); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Elasticsearch7DynamicSink that = (Elasticsearch7DynamicSink) o; + return Objects.equals(format, that.format) + && Objects.equals(schema, that.schema) + && Objects.equals(config, that.config) + && Objects.equals(builderProvider, that.builderProvider); + } + + @Override + public int hashCode() { + return Objects.hash(format, schema, config, builderProvider); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java new file mode 100644 index 00000000..50c37f07 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -0,0 +1,186 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.StringUtils; + +import java.time.ZoneId; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FORMAT_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.HOSTS_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.INDEX_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.KEY_DELIMITER_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.PASSWORD_OPTION; +import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.USERNAME_OPTION; + +/** A {@link DynamicTableSinkFactory} for discovering {@link Elasticsearch7DynamicSink}. */ +@Internal +public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory { + private static final Set> requiredOptions = + Stream.of(HOSTS_OPTION, INDEX_OPTION).collect(Collectors.toSet()); + private static final Set> optionalOptions = + Stream.of( + KEY_DELIMITER_OPTION, + FAILURE_HANDLER_OPTION, + FLUSH_ON_CHECKPOINT_OPTION, + BULK_FLASH_MAX_SIZE_OPTION, + BULK_FLUSH_MAX_ACTIONS_OPTION, + BULK_FLUSH_INTERVAL_OPTION, + BULK_FLUSH_BACKOFF_TYPE_OPTION, + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, + BULK_FLUSH_BACKOFF_DELAY_OPTION, + CONNECTION_MAX_RETRY_TIMEOUT_OPTION, + CONNECTION_PATH_PREFIX, + FORMAT_OPTION, + PASSWORD_OPTION, + USERNAME_OPTION) + .collect(Collectors.toSet()); + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + TableSchema tableSchema = context.getCatalogTable().getSchema(); + ElasticsearchValidationUtils.validatePrimaryKey(tableSchema); + + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + + final EncodingFormat> format = + helper.discoverEncodingFormat(SerializationFormatFactory.class, FORMAT_OPTION); + + helper.validate(); + Configuration configuration = new Configuration(); + context.getCatalogTable().getOptions().forEach(configuration::setString); + Elasticsearch7Configuration config = + new Elasticsearch7Configuration(configuration, context.getClassLoader()); + + validate(config, configuration); + + return new Elasticsearch7DynamicSink( + format, + config, + TableSchemaUtils.getPhysicalSchema(tableSchema), + getLocalTimeZoneId(context.getConfiguration())); + } + + ZoneId getLocalTimeZoneId(ReadableConfig readableConfig) { + final String zone = readableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE); + final ZoneId zoneId = + TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); + + return zoneId; + } + + private void validate(Elasticsearch7Configuration config, Configuration originalConfiguration) { + config.getFailureHandler(); // checks if we can instantiate the custom failure handler + config.getHosts(); // validate hosts + validate( + config.getIndex().length() >= 1, + () -> String.format("'%s' must not be empty", INDEX_OPTION.key())); + int maxActions = config.getBulkFlushMaxActions(); + validate( + maxActions == -1 || maxActions >= 1, + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_MAX_ACTIONS_OPTION.key(), maxActions)); + long maxSize = config.getBulkFlushMaxByteSize(); + long mb1 = 1024 * 1024; + validate( + maxSize == -1 || (maxSize >= mb1 && maxSize % mb1 == 0), + () -> + String.format( + "'%s' must be in MB granularity. Got: %s", + BULK_FLASH_MAX_SIZE_OPTION.key(), + originalConfiguration + .get(BULK_FLASH_MAX_SIZE_OPTION) + .toHumanReadableString())); + validate( + config.getBulkFlushBackoffRetries().map(retries -> retries >= 1).orElse(true), + () -> + String.format( + "'%s' must be at least 1. Got: %s", + BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), + config.getBulkFlushBackoffRetries().get())); + if (config.getUsername().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getUsername().get())) { + validate( + config.getPassword().isPresent() + && !StringUtils.isNullOrWhitespaceOnly(config.getPassword().get()), + () -> + String.format( + "'%s' and '%s' must be set at the same time. Got: username '%s' and password '%s'", + USERNAME_OPTION.key(), + PASSWORD_OPTION.key(), + config.getUsername().get(), + config.getPassword().orElse(""))); + } + } + + private static void validate(boolean condition, Supplier message) { + if (!condition) { + throw new ValidationException(message.get()); + } + } + + @Override + public String factoryIdentifier() { + return "elasticsearch-7"; + } + + @Override + public Set> requiredOptions() { + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + return optionalOptions; + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index 9e189e38..10e4846e 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -13,4 +13,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -org.apache.flink.connector.elasticsearch.table.Elasticsearch7DynamicSinkFactory +org.apache.flink.streaming.connectors.elasticsearch.table.Elasticsearch7DynamicSinkFactory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java new file mode 100644 index 00000000..3ff21ed1 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java @@ -0,0 +1,234 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.typeutils.base.VoidSerializer; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.util.TestLogger; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Arrays; +import java.util.Collections; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; + +/** Tests for validation in {@link Elasticsearch7DynamicSinkFactory}. */ +public class Elasticsearch7DynamicSinkFactoryTest extends TestLogger { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void validateEmptyConfiguration() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "One or more required options are missing.\n" + + "\n" + + "Missing required options are:\n" + + "\n" + + "hosts\n" + + "index"); + sinkFactory.createDynamicTableSink(context().build()); + } + + @Test + public void validateWrongIndex() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'index' must not be empty"); + sinkFactory.createDynamicTableSink( + context() + .withOption("index", "") + .withOption("hosts", "http://localhost:12345") + .build()); + } + + @Test + public void validateWrongHosts() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "Could not parse host 'wrong-host' in option 'hosts'. It should follow the format 'http://host_name:port'."); + sinkFactory.createDynamicTableSink( + context().withOption("index", "MyIndex").withOption("hosts", "wrong-host").build()); + } + + @Test + public void validateWrongFlushSize() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'sink.bulk-flush.max-size' must be in MB granularity. Got: 1024 bytes"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), + "1kb") + .build()); + } + + @Test + public void validateWrongRetries() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.backoff.max-retries' must be at least 1. Got: 0"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION + .key(), + "0") + .build()); + } + + @Test + public void validateWrongMaxActions() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("'sink.bulk-flush.max-actions' must be at least 1. Got: -2"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), + "-2") + .build()); + } + + @Test + public void validateWrongBackoffDelay() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage("Invalid value for option 'sink.bulk-flush.backoff.delay'."); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "-1s") + .build()); + } + + @Test + public void validatePrimaryKeyOnIllegalColumn() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "The table has a primary key on columns of illegal types: " + + "[ARRAY, MAP, MULTISET, ROW, RAW, VARBINARY].\n" + + " Elasticsearch sink does not support primary keys on columns of types: " + + "[ARRAY, MAP, MULTISET, STRUCTURED_TYPE, ROW, RAW, BINARY, VARBINARY]."); + sinkFactory.createDynamicTableSink( + context() + .withSchema( + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical( + "b", + DataTypes.ARRAY( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "c", + DataTypes.MAP( + DataTypes.BIGINT(), + DataTypes.STRING()) + .notNull()), + Column.physical( + "d", + DataTypes.MULTISET( + DataTypes.BIGINT() + .notNull()) + .notNull()), + Column.physical( + "e", + DataTypes.ROW( + DataTypes.FIELD( + "a", + DataTypes.BIGINT())) + .notNull()), + Column.physical( + "f", + DataTypes.RAW( + Void.class, + VoidSerializer.INSTANCE) + .notNull()), + Column.physical("g", DataTypes.BYTES().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey( + "name", + Arrays.asList("a", "b", "c", "d", "e", "f", "g")))) + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), + "1s") + .build()); + } + + @Test + public void validateWrongCredential() { + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + thrown.expect(ValidationException.class); + thrown.expectMessage( + "'username' and 'password' must be set at the same time. Got: username 'username' and password ''"); + sinkFactory.createDynamicTableSink( + context() + .withOption(ElasticsearchConnectorOptions.INDEX_OPTION.key(), "MyIndex") + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + "http://localhost:1234") + .withOption(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), "username") + .withOption(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), "") + .build()); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java new file mode 100644 index 00000000..fdf7a3a0 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -0,0 +1,376 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.DockerImageVersions; +import org.apache.flink.util.TestLogger; + +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.transport.client.PreBuiltTransportClient; +import org.junit.ClassRule; +import org.junit.Test; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.time.Duration; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import static org.apache.flink.streaming.connectors.elasticsearch.table.TestContext.context; +import static org.apache.flink.table.api.Expressions.row; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.junit.Assert.assertThat; + +/** IT tests for {@link Elasticsearch7DynamicSink}. */ +public class Elasticsearch7DynamicSinkITCase extends TestLogger { + + @ClassRule + public static ElasticsearchContainer elasticsearchContainer = + new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); + + @SuppressWarnings("deprecation") + protected final Client getClient() { + TransportAddress transportAddress = + new TransportAddress(elasticsearchContainer.getTcpHost()); + String expectedClusterName = "docker-cluster"; + Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); + return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + } + + @Test + public void testWritingDocuments() throws Exception { + ResolvedSchema schema = + new ResolvedSchema( + Arrays.asList( + Column.physical("a", DataTypes.BIGINT().notNull()), + Column.physical("b", DataTypes.TIME()), + Column.physical("c", DataTypes.STRING().notNull()), + Column.physical("d", DataTypes.FLOAT()), + Column.physical("e", DataTypes.TINYINT().notNull()), + Column.physical("f", DataTypes.DATE()), + Column.physical("g", DataTypes.TIMESTAMP().notNull())), + Collections.emptyList(), + UniqueConstraint.primaryKey("name", Arrays.asList("a", "g"))); + + GenericRowData rowData = + GenericRowData.of( + 1L, + 12345, + StringData.fromString("ABCDE"), + 12.12f, + (byte) 2, + 12345, + TimestampData.fromLocalDateTime( + LocalDateTime.parse("2012-12-12T12:12:12"))); + + String index = "writing-documents"; + Elasticsearch7DynamicSinkFactory sinkFactory = new Elasticsearch7DynamicSinkFactory(); + + SinkFunctionProvider sinkRuntimeProvider = + (SinkFunctionProvider) + sinkFactory + .createDynamicTableSink( + context() + .withSchema(schema) + .withOption( + ElasticsearchConnectorOptions.INDEX_OPTION + .key(), + index) + .withOption( + ElasticsearchConnectorOptions.HOSTS_OPTION + .key(), + elasticsearchContainer.getHttpHostAddress()) + .withOption( + ElasticsearchConnectorOptions + .FLUSH_ON_CHECKPOINT_OPTION + .key(), + "false") + .build()) + .getSinkRuntimeProvider(new MockContext()); + + SinkFunction sinkFunction = sinkRuntimeProvider.createSinkFunction(); + StreamExecutionEnvironment environment = + StreamExecutionEnvironment.getExecutionEnvironment(); + environment.setParallelism(4); + + rowData.setRowKind(RowKind.UPDATE_AFTER); + environment.fromElements(rowData).addSink(sinkFunction); + environment.execute(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsFromTableApi() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "table-api"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL," + + "h as a + 2,\n" + + "PRIMARY KEY (a, g) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "00:00:12"); + expectedMap.put("c", "ABCDE"); + expectedMap.put("d", 12.12d); + expectedMap.put("e", 2); + expectedMap.put("f", "2003-10-20"); + expectedMap.put("g", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + @Test + public void testWritingDocumentsNoPrimaryKey() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "no-primary-key"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIME,\n" + + "c STRING NOT NULL,\n" + + "d FLOAT,\n" + + "e TINYINT NOT NULL,\n" + + "f DATE,\n" + + "g TIMESTAMP NOT NULL\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues( + row( + 1L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "ABCDE", + 12.12f, + (byte) 2, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2012-12-12T12:12:12")), + row( + 2L, + LocalTime.ofNanoOfDay(12345L * 1_000_000L), + "FGHIJK", + 13.13f, + (byte) 4, + LocalDate.ofEpochDay(12345), + LocalDateTime.parse("2013-12-12T13:13:13"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + + // search API does not return documents that were not indexed, we might need to query + // the index a few times + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + SearchHits hits; + do { + hits = client.prepareSearch(index).execute().actionGet().getHits(); + if (hits.getTotalHits().value < 2) { + Thread.sleep(200); + } + } while (hits.getTotalHits().value < 2 && deadline.hasTimeLeft()); + + if (hits.getTotalHits().value < 2) { + throw new AssertionError("Could not retrieve results from Elasticsearch."); + } + + HashSet> resultSet = new HashSet<>(); + resultSet.add(hits.getAt(0).getSourceAsMap()); + resultSet.add(hits.getAt(1).getSourceAsMap()); + Map expectedMap1 = new HashMap<>(); + expectedMap1.put("a", 1); + expectedMap1.put("b", "00:00:12"); + expectedMap1.put("c", "ABCDE"); + expectedMap1.put("d", 12.12d); + expectedMap1.put("e", 2); + expectedMap1.put("f", "2003-10-20"); + expectedMap1.put("g", "2012-12-12 12:12:12"); + Map expectedMap2 = new HashMap<>(); + expectedMap2.put("a", 2); + expectedMap2.put("b", "00:00:12"); + expectedMap2.put("c", "FGHIJK"); + expectedMap2.put("d", 13.13d); + expectedMap2.put("e", 4); + expectedMap2.put("f", "2003-10-20"); + expectedMap2.put("g", "2013-12-12 13:13:13"); + HashSet> expectedSet = new HashSet<>(); + expectedSet.add(expectedMap1); + expectedSet.add(expectedMap2); + assertThat(resultSet, equalTo(expectedSet)); + } + + @Test + public void testWritingDocumentsWithDynamicIndex() throws Exception { + TableEnvironment tableEnvironment = + TableEnvironment.create(EnvironmentSettings.inStreamingMode()); + + String index = "dynamic-index-{b|yyyy-MM-dd}"; + tableEnvironment.executeSql( + "CREATE TABLE esTable (" + + "a BIGINT NOT NULL,\n" + + "b TIMESTAMP NOT NULL,\n" + + "PRIMARY KEY (a) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + String.format("'%s'='%s',\n", "connector", "elasticsearch-7") + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.INDEX_OPTION.key(), index) + + String.format( + "'%s'='%s',\n", + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + elasticsearchContainer.getHttpHostAddress()) + + String.format( + "'%s'='%s'\n", + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), + "false") + + ")"); + + tableEnvironment + .fromValues(row(1L, LocalDateTime.parse("2012-12-12T12:12:12"))) + .executeInsert("esTable") + .await(); + + Client client = getClient(); + Map response = + client.get(new GetRequest("dynamic-index-2012-12-12", "1")).actionGet().getSource(); + Map expectedMap = new HashMap<>(); + expectedMap.put("a", 1); + expectedMap.put("b", "2012-12-12 12:12:12"); + assertThat(response, equalTo(expectedMap)); + } + + private static class MockContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } +} diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java new file mode 100644 index 00000000..2928c8db --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java @@ -0,0 +1,298 @@ +/* + * 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.connectors.elasticsearch.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch7.ElasticsearchSink; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.TestLogger; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; +import java.util.List; + +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +/** Tests for {@link Elasticsearch7DynamicSink} parameters. */ +public class Elasticsearch7DynamicSinkTest extends TestLogger { + + private static final String FIELD_KEY = "key"; + private static final String FIELD_FRUIT_NAME = "fruit_name"; + private static final String FIELD_COUNT = "count"; + private static final String FIELD_TS = "ts"; + + private static final String HOSTNAME = "host1"; + private static final int PORT = 1234; + private static final String SCHEMA = "https"; + private static final String INDEX = "MyIndex"; + private static final String DOC_TYPE = "MyType"; + private static final String USERNAME = "username"; + private static final String PASSWORD = "password"; + + @Test + public void testBuilder() { + final TableSchema schema = createTestSchema(); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + getConfig(), this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new DummyFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(true); + verify(provider.builderSpy) + .setBulkFlushBackoffType(ElasticsearchSinkBase.FlushBackoffType.EXPONENTIAL); + verify(provider.builderSpy).setBulkFlushBackoffDelay(123); + verify(provider.builderSpy).setBulkFlushBackoffRetries(3); + verify(provider.builderSpy).setBulkFlushInterval(100); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(1); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.DefaultRestClientFactory("/myapp")); + verify(provider.sinkSpy).disableFlushOnCheckpoint(); + } + + @Test + public void testDefaultConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory(new Elasticsearch7DynamicSink.DefaultRestClientFactory(null)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + @Test + public void testAuthConfig() { + final TableSchema schema = createTestSchema(); + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString(ElasticsearchConnectorOptions.USERNAME_OPTION.key(), USERNAME); + configuration.setString(ElasticsearchConnectorOptions.PASSWORD_OPTION.key(), PASSWORD); + + BuilderProvider provider = new BuilderProvider(); + final Elasticsearch7DynamicSink testSink = + new Elasticsearch7DynamicSink( + new DummyEncodingFormat(), + new Elasticsearch7Configuration( + configuration, this.getClass().getClassLoader()), + schema, + ZoneId.systemDefault(), + provider); + + testSink.getSinkRuntimeProvider(new MockSinkContext()).createSinkFunction(); + + verify(provider.builderSpy).setFailureHandler(new NoOpFailureHandler()); + verify(provider.builderSpy).setBulkFlushBackoff(false); + verify(provider.builderSpy).setBulkFlushInterval(1000); + verify(provider.builderSpy).setBulkFlushMaxActions(1000); + verify(provider.builderSpy).setBulkFlushMaxSizeMb(2); + verify(provider.builderSpy) + .setRestClientFactory( + new Elasticsearch7DynamicSink.AuthRestClientFactory( + null, USERNAME, PASSWORD)); + verify(provider.sinkSpy, never()).disableFlushOnCheckpoint(); + } + + private Configuration getConfig() { + Configuration configuration = new Configuration(); + configuration.setString(ElasticsearchConnectorOptions.INDEX_OPTION.key(), INDEX); + configuration.setString(ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION.key(), DOC_TYPE); + configuration.setString( + ElasticsearchConnectorOptions.HOSTS_OPTION.key(), + SCHEMA + "://" + HOSTNAME + ":" + PORT); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION.key(), "exponential"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_DELAY_OPTION.key(), "123"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION.key(), "3"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION.key(), "100"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION.key(), "1000"); + configuration.setString( + ElasticsearchConnectorOptions.BULK_FLASH_MAX_SIZE_OPTION.key(), "1mb"); + configuration.setString( + ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX.key(), "/myapp"); + configuration.setString( + ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION.key(), + DummyFailureHandler.class.getName()); + configuration.setString( + ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION.key(), "false"); + return configuration; + } + + private static class BuilderProvider + implements Elasticsearch7DynamicSink.ElasticSearchBuilderProvider { + public ElasticsearchSink.Builder builderSpy; + public ElasticsearchSink sinkSpy; + + @Override + public ElasticsearchSink.Builder createBuilder( + List httpHosts, RowElasticsearchSinkFunction upsertSinkFunction) { + builderSpy = + Mockito.spy(new ElasticsearchSink.Builder<>(httpHosts, upsertSinkFunction)); + doAnswer( + invocation -> { + sinkSpy = + Mockito.spy( + (ElasticsearchSink) + invocation.callRealMethod()); + return sinkSpy; + }) + .when(builderSpy) + .build(); + + return builderSpy; + } + } + + private TableSchema createTestSchema() { + return TableSchema.builder() + .field(FIELD_KEY, DataTypes.BIGINT()) + .field(FIELD_FRUIT_NAME, DataTypes.STRING()) + .field(FIELD_COUNT, DataTypes.DECIMAL(10, 4)) + .field(FIELD_TS, DataTypes.TIMESTAMP(3)) + .build(); + } + + private static class DummySerializationSchema implements SerializationSchema { + + private static final DummySerializationSchema INSTANCE = new DummySerializationSchema(); + + @Override + public byte[] serialize(RowData element) { + return new byte[0]; + } + } + + private static class DummyEncodingFormat + implements EncodingFormat> { + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return DummySerializationSchema.INSTANCE; + } + + @Override + public ChangelogMode getChangelogMode() { + return null; + } + } + + private static class MockSinkContext implements DynamicTableSink.Context { + @Override + public boolean isBounded() { + return false; + } + + @Override + public TypeInformation createTypeInformation(DataType consumedDataType) { + return null; + } + + @Override + public TypeInformation createTypeInformation(LogicalType consumedLogicalType) { + return null; + } + + @Override + public DynamicTableSink.DataStructureConverter createDataStructureConverter( + DataType consumedDataType) { + return null; + } + } + + /** Custom failure handler for testing. */ + public static class DummyFailureHandler implements ActionRequestFailureHandler { + + @Override + public void onFailure( + ActionRequest action, + Throwable failure, + int restStatusCode, + RequestIndexer indexer) { + // do nothing + } + + @Override + public boolean equals(Object o) { + return o instanceof DummyFailureHandler; + } + + @Override + public int hashCode() { + return DummyFailureHandler.class.hashCode(); + } + } +} From 9daaddae547e1ad0d266b65339d4025a8001dd88 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Mon, 21 Mar 2022 16:10:30 +0200 Subject: [PATCH 200/207] [FLINK-26709][table] Replace TableConfig.getConfiguration.set() Since `TableConfig` is a `WritableConfig`, callers should directly call `TableConfig.set()` and avoid going throught `#getConfiguration()` which is there only for advanced internal configuration read value purposes. --- .../table/ElasticsearchDynamicSinkBaseITCase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java index 21ad5fd2..75dd9b98 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UniqueConstraint; @@ -310,10 +311,7 @@ public void testWritingDocumentsWithDynamicIndexFromSystemTime() throws Exceptio TableEnvironment.create(EnvironmentSettings.inStreamingMode()); DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd"); - tableEnvironment - .getConfig() - .getConfiguration() - .setString("table.local-time-zone", "Asia/Shanghai"); + tableEnvironment.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "Asia/Shanghai"); String dynamicIndex1 = "dynamic-index-" From 37a898621565ee1c71f8d418d0f1f300589992d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 22 Mar 2022 14:48:49 +0100 Subject: [PATCH 201/207] [FLINK-26281][connectors/elasticsearch] Remove unused 'connection.max-retry-timeout' option This closes #19202. --- .../elasticsearch/table/ElasticsearchConnectorOptions.java | 6 ------ .../table/Elasticsearch6DynamicSinkFactory.java | 2 -- .../table/Elasticsearch7DynamicSinkFactory.java | 2 -- 3 files changed, 10 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 1b0fc5e4..38d65a7f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -133,12 +133,6 @@ public class ElasticsearchConnectorOptions { .noDefaultValue() .withDescription("Delay between each backoff attempt."); - public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = - ConfigOptions.key("connection.max-retry-timeout") - .durationType() - .noDefaultValue() - .withDescription("Maximum timeout between retries."); - public static final ConfigOption CONNECTION_PATH_PREFIX = ConfigOptions.key("connection.path-prefix") .stringType() diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index ff600a7a..73f5cf88 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -47,7 +47,6 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; @@ -75,7 +74,6 @@ public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_TYPE_OPTION, BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, FORMAT_OPTION, PASSWORD_OPTION, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 50c37f07..70b0134d 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -47,7 +47,6 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; @@ -74,7 +73,6 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_TYPE_OPTION, BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, FORMAT_OPTION, PASSWORD_OPTION, From a831407ba44b8a5b79494d119767741a27372e48 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 4 Feb 2022 10:44:57 -0500 Subject: [PATCH 202/207] [FLINK-25961] [connectors/elasticsearch] Remove transport client from Elasticsearch 6/7 connectors (tests only) Signed-off-by: Andriy Redko --- .../pom.xml | 8 ---- .../ElasticsearchSinkTestBase.java | 8 ++-- .../testutils/SourceSinkDataTestKit.java | 10 +++-- .../flink-connector-elasticsearch6/pom.xml | 18 -------- .../Elasticsearch6DynamicSinkITCase.java | 43 ++++++++++--------- .../ElasticsearchSinkITCase.java | 15 ++----- .../flink-connector-elasticsearch7/pom.xml | 18 -------- .../Elasticsearch7DynamicSinkITCase.java | 37 ++++++++-------- .../ElasticsearchSinkITCase.java | 15 ++----- 9 files changed, 59 insertions(+), 113 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index a7417663..89ae167b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -96,14 +96,6 @@ under the License. - - - org.elasticsearch.client - transport - ${elasticsearch.version} - test - - org.testcontainers elasticsearch diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 1d250b7e..84342fc6 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.flink.test.util.AbstractTestBase; -import org.elasticsearch.client.Client; +import org.elasticsearch.client.RestHighLevelClient; import java.util.Collections; import java.util.HashMap; @@ -44,9 +44,7 @@ public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { - // It's not good that we're using a Client here instead of a Rest Client but we need this - // for compatibility with ES 5.3.x. As soon as we drop that we can use RestClient here. - protected abstract Client getClient(); + protected abstract RestHighLevelClient getClient(); protected abstract String getClusterName(); @@ -90,7 +88,7 @@ private void runElasticSearchSinkTest( env.execute("Elasticsearch Sink Test"); // verify the results - Client client = getClient(); + RestHighLevelClient client = getClient(); SourceSinkDataTestKit.verifyProducedSinkData(client, index); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java index 41e199de..584ed4d3 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java @@ -26,7 +26,8 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.client.Client; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.junit.Assert; @@ -136,10 +137,13 @@ public void process( * @param client The client to use to connect to Elasticsearch * @param index The index to check */ - public static void verifyProducedSinkData(Client client, String index) { + public static void verifyProducedSinkData(RestHighLevelClient client, String index) + throws IOException { for (int i = 0; i < NUM_ELEMENTS; i++) { GetResponse response = - client.get(new GetRequest(index, TYPE_NAME, Integer.toString(i))).actionGet(); + client.get( + new GetRequest(index, TYPE_NAME, Integer.toString(i)), + RequestOptions.DEFAULT); Assert.assertEquals(DATA_PREFIX + i, response.getSource().get(DATA_FIELD_NAME)); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 3af8f666..ad428def 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -139,24 +139,6 @@ under the License. test - - - - org.elasticsearch.client - transport - ${elasticsearch.version} - test - - - - org.elasticsearch.plugin - transport-netty4-client - ${elasticsearch.version} - test - - org.apache.logging.log4j log4j-api diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java index 0ebc52b5..6717e8fa 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java @@ -40,12 +40,13 @@ import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; +import org.apache.http.HttpHost; import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; import org.testcontainers.elasticsearch.ElasticsearchContainer; @@ -74,12 +75,9 @@ public class Elasticsearch6DynamicSinkITCase extends TestLogger { new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_6)); @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } @Test @@ -147,10 +145,11 @@ public void testWritingDocuments() throws Exception { environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) - .actionGet() + client.get( + new GetRequest(index, myType, "1_2012-12-12T12:12:12"), + RequestOptions.DEFAULT) .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); @@ -213,10 +212,11 @@ public void testWritingDocumentsFromTableApi() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest(index, myType, "1_2012-12-12T12:12:12")) - .actionGet() + client.get( + new GetRequest(index, myType, "1_2012-12-12T12:12:12"), + RequestOptions.DEFAULT) .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); @@ -285,14 +285,14 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); // search API does not return documents that were not indexed, we might need to query // the index a few times Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); SearchHits hits; do { - hits = client.prepareSearch(index).execute().actionGet().getHits(); + hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); if (hits.getTotalHits() < 2) { Thread.sleep(200); } @@ -363,10 +363,11 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest("dynamic-index-2012-12-12", myType, "1")) - .actionGet() + client.get( + new GetRequest("dynamic-index-2012-12-12", myType, "1"), + RequestOptions.DEFAULT) .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index b29e9585..c2fd9f8d 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -26,11 +26,8 @@ import org.apache.flink.util.DockerImageVersions; import org.apache.http.HttpHost; -import org.elasticsearch.client.Client; +import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; @@ -57,13 +54,9 @@ protected String getClusterName() { } @Override - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = getClusterName(); - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } @Test diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 3c4bb4f7..67348d42 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -142,24 +142,6 @@ under the License. test - - - - org.elasticsearch.client - transport - ${elasticsearch.version} - test - - - - org.elasticsearch.plugin - transport-netty4-client - ${elasticsearch.version} - test - - org.apache.flink diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java index fdf7a3a0..3534f4a8 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java @@ -40,12 +40,13 @@ import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; +import org.apache.http.HttpHost; import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; import org.testcontainers.elasticsearch.ElasticsearchContainer; @@ -74,12 +75,9 @@ public class Elasticsearch7DynamicSinkITCase extends TestLogger { new ElasticsearchContainer(DockerImageName.parse(DockerImageVersions.ELASTICSEARCH_7)); @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = "docker-cluster"; - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } @Test @@ -142,9 +140,10 @@ public void testWritingDocuments() throws Exception { environment.fromElements(rowData).addSink(sinkFunction); environment.execute(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); expectedMap.put("b", "00:00:12"); @@ -202,9 +201,10 @@ public void testWritingDocumentsFromTableApi() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest(index, "1_2012-12-12T12:12:12")).actionGet().getSource(); + client.get(new GetRequest(index, "1_2012-12-12T12:12:12"), RequestOptions.DEFAULT) + .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); expectedMap.put("b", "00:00:12"); @@ -268,14 +268,14 @@ public void testWritingDocumentsNoPrimaryKey() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); // search API does not return documents that were not indexed, we might need to query // the index a few times Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); SearchHits hits; do { - hits = client.prepareSearch(index).execute().actionGet().getHits(); + hits = client.search(new SearchRequest(index), RequestOptions.DEFAULT).getHits(); if (hits.getTotalHits().value < 2) { Thread.sleep(200); } @@ -342,9 +342,10 @@ public void testWritingDocumentsWithDynamicIndex() throws Exception { .executeInsert("esTable") .await(); - Client client = getClient(); + RestHighLevelClient client = getClient(); Map response = - client.get(new GetRequest("dynamic-index-2012-12-12", "1")).actionGet().getSource(); + client.get(new GetRequest("dynamic-index-2012-12-12", "1"), RequestOptions.DEFAULT) + .getSource(); Map expectedMap = new HashMap<>(); expectedMap.put("a", 1); expectedMap.put("b", "2012-12-12 12:12:12"); diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java index e5f89bd0..e5a5527c 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java @@ -26,11 +26,8 @@ import org.apache.flink.util.DockerImageVersions; import org.apache.http.HttpHost; -import org.elasticsearch.client.Client; +import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.transport.client.PreBuiltTransportClient; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; @@ -57,13 +54,9 @@ protected String getClusterName() { } @Override - @SuppressWarnings("deprecation") - protected final Client getClient() { - TransportAddress transportAddress = - new TransportAddress(elasticsearchContainer.getTcpHost()); - String expectedClusterName = getClusterName(); - Settings settings = Settings.builder().put("cluster.name", expectedClusterName).build(); - return new PreBuiltTransportClient(settings).addTransportAddress(transportAddress); + protected final RestHighLevelClient getClient() { + return new RestHighLevelClient( + RestClient.builder(HttpHost.create(elasticsearchContainer.getHttpHostAddress()))); } @Test From 91fc0a8749d923c599ac3c5ba890e17466364789 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Fri, 1 Apr 2022 12:34:02 +0200 Subject: [PATCH 203/207] [FLINK-26884] add files about licence, gitingore, editorconfig. --- .editorconfig | 276 ++++++++++++++++++++++++++++++++++++++++++++++++++ .gitignore | 38 +++++++ LICENSE | 201 ++++++++++++++++++++++++++++++++++++ NOTICE | 17 ++++ 4 files changed, 532 insertions(+) create mode 100644 .editorconfig create mode 100644 .gitignore create mode 100644 LICENSE create mode 100644 NOTICE diff --git a/.editorconfig b/.editorconfig new file mode 100644 index 00000000..2e260749 --- /dev/null +++ b/.editorconfig @@ -0,0 +1,276 @@ +root = true + +[*] +charset = utf-8 +end_of_line = lf +insert_final_newline = true +max_line_length = 100 +# ij_formatter_off_tag = @formatter:off +# ij_formatter_on_tag = @formatter:on +# ij_formatter_tags_enabled = false +# ij_smart_tabs = false +# ij_wrap_on_typing = false + +[*.java] +indent_size = 4 +indent_style = space +tab_width = 4 +ij_continuation_indent_size = 8 +# ij_java_align_consecutive_assignments = false +# ij_java_align_consecutive_variable_declarations = false +# ij_java_align_group_field_declarations = false +# ij_java_align_multiline_annotation_parameters = false +# ij_java_align_multiline_array_initializer_expression = false +# ij_java_align_multiline_assignment = false +# ij_java_align_multiline_binary_operation = false +# ij_java_align_multiline_chained_methods = false +# ij_java_align_multiline_extends_list = false +# ij_java_align_multiline_for = true +# ij_java_align_multiline_method_parentheses = false +# ij_java_align_multiline_parameters = true +# ij_java_align_multiline_parameters_in_calls = false +# ij_java_align_multiline_parenthesized_expression = false +# ij_java_align_multiline_records = true +# ij_java_align_multiline_resources = true +# ij_java_align_multiline_ternary_operation = false +# ij_java_align_multiline_text_blocks = false +# ij_java_align_multiline_throws_list = false +# ij_java_align_subsequent_simple_methods = false +# ij_java_align_throws_keyword = false +# ij_java_annotation_parameter_wrap = off +# ij_java_array_initializer_new_line_after_left_brace = false +# ij_java_array_initializer_right_brace_on_new_line = false +# ij_java_array_initializer_wrap = off +# ij_java_assert_statement_colon_on_next_line = false +# ij_java_assert_statement_wrap = off +# ij_java_assignment_wrap = off +ij_java_binary_operation_sign_on_next_line = true +ij_java_binary_operation_wrap = normal +# ij_java_blank_lines_after_anonymous_class_header = 0 +# ij_java_blank_lines_after_class_header = 0 +# ij_java_blank_lines_after_imports = 1 +# ij_java_blank_lines_after_package = 1 +# ij_java_blank_lines_around_class = 1 +# ij_java_blank_lines_around_field = 0 +# ij_java_blank_lines_around_field_in_interface = 0 +# ij_java_blank_lines_around_initializer = 1 +# ij_java_blank_lines_around_method = 1 +# ij_java_blank_lines_around_method_in_interface = 1 +# ij_java_blank_lines_before_class_end = 0 +# ij_java_blank_lines_before_imports = 1 +# ij_java_blank_lines_before_method_body = 0 +# ij_java_blank_lines_before_package = 0 +# ij_java_block_brace_style = end_of_line +# ij_java_block_comment_at_first_column = true +ij_java_call_parameters_new_line_after_left_paren = true +# ij_java_call_parameters_right_paren_on_new_line = false +ij_java_call_parameters_wrap = on_every_item +# ij_java_case_statement_on_separate_line = true +# ij_java_catch_on_new_line = false +# ij_java_class_annotation_wrap = split_into_lines +# ij_java_class_brace_style = end_of_line +ij_java_class_count_to_use_import_on_demand = 9999 +# ij_java_class_names_in_javadoc = 1 +# ij_java_do_not_indent_top_level_class_members = false +# ij_java_do_not_wrap_after_single_annotation = false +# ij_java_do_while_brace_force = never +# ij_java_doc_add_blank_line_after_description = true +ij_java_doc_add_blank_line_after_param_comments = true +ij_java_doc_add_blank_line_after_return = true +# ij_java_doc_add_p_tag_on_empty_lines = true +ij_java_doc_align_exception_comments = false +ij_java_doc_align_param_comments = false +ij_java_doc_do_not_wrap_if_one_line = true +ij_java_doc_enable_formatting = true +# ij_java_doc_enable_leading_asterisks = true +ij_java_doc_indent_on_continuation = true +ij_java_doc_keep_empty_lines = true +# ij_java_doc_keep_empty_parameter_tag = true +# ij_java_doc_keep_empty_return_tag = true +# ij_java_doc_keep_empty_throws_tag = true +# ij_java_doc_keep_invalid_tags = true +# ij_java_doc_param_description_on_new_line = false +ij_java_doc_preserve_line_breaks = false +# ij_java_doc_use_throws_not_exception_tag = true +# ij_java_else_on_new_line = false +# ij_java_entity_dd_suffix = EJB +# ij_java_entity_eb_suffix = Bean +# ij_java_entity_hi_suffix = Home +# ij_java_entity_lhi_prefix = Local +# ij_java_entity_lhi_suffix = Home +# ij_java_entity_li_prefix = Local +# ij_java_entity_pk_class = java.lang.String +# ij_java_entity_vo_suffix = VO +# ij_java_enum_constants_wrap = off +# ij_java_extends_keyword_wrap = off +# ij_java_extends_list_wrap = off +# ij_java_field_annotation_wrap = split_into_lines +# ij_java_finally_on_new_line = false +# ij_java_for_brace_force = never +# ij_java_for_statement_new_line_after_left_paren = false +# ij_java_for_statement_right_paren_on_new_line = false +# ij_java_for_statement_wrap = off +# ij_java_generate_final_locals = false +# ij_java_generate_final_parameters = false +# ij_java_if_brace_force = never +ij_java_imports_layout = org.apache.flink.**,|,org.apache.flink.shaded.**,|,*,|,javax.**,|,java.**,|,scala.**,|,$* +# ij_java_indent_case_from_switch = true +# ij_java_insert_inner_class_imports = false +# ij_java_insert_override_annotation = true +# ij_java_keep_blank_lines_before_right_brace = 2 +# ij_java_keep_blank_lines_between_package_declaration_and_header = 2 +# ij_java_keep_blank_lines_in_code = 2 +# ij_java_keep_blank_lines_in_declarations = 2 +# ij_java_keep_control_statement_in_one_line = true +# ij_java_keep_first_column_comment = true +# ij_java_keep_indents_on_empty_lines = false +# ij_java_keep_line_breaks = true +# ij_java_keep_multiple_expressions_in_one_line = false +# ij_java_keep_simple_blocks_in_one_line = false +# ij_java_keep_simple_classes_in_one_line = false +# ij_java_keep_simple_lambdas_in_one_line = false +# ij_java_keep_simple_methods_in_one_line = false +# ij_java_label_indent_absolute = false +# ij_java_label_indent_size = 0 +# ij_java_lambda_brace_style = end_of_line +ij_java_layout_static_imports_separately = true +# ij_java_line_comment_add_space = false +# ij_java_line_comment_at_first_column = true +# ij_java_message_dd_suffix = EJB +# ij_java_message_eb_suffix = Bean +# ij_java_method_annotation_wrap = split_into_lines +# ij_java_method_brace_style = end_of_line +ij_java_method_call_chain_wrap = on_every_item +ij_java_method_parameters_new_line_after_left_paren = true +# ij_java_method_parameters_right_paren_on_new_line = false +ij_java_method_parameters_wrap = on_every_item +# ij_java_modifier_list_wrap = false +ij_java_names_count_to_use_import_on_demand = 9999 +# ij_java_new_line_after_lparen_in_record_header = false +# ij_java_packages_to_use_import_on_demand = java.awt.*,javax.swing.* +# ij_java_parameter_annotation_wrap = off +# ij_java_parentheses_expression_new_line_after_left_paren = false +# ij_java_parentheses_expression_right_paren_on_new_line = false +# ij_java_place_assignment_sign_on_next_line = false +# ij_java_prefer_longer_names = true +# ij_java_prefer_parameters_wrap = false +# ij_java_record_components_wrap = normal +# ij_java_repeat_synchronized = true +# ij_java_replace_instanceof_and_cast = false +# ij_java_replace_null_check = true +# ij_java_replace_sum_lambda_with_method_ref = true +# ij_java_resource_list_new_line_after_left_paren = false +# ij_java_resource_list_right_paren_on_new_line = false +# ij_java_resource_list_wrap = off +# ij_java_rparen_on_new_line_in_record_header = false +# ij_java_session_dd_suffix = EJB +# ij_java_session_eb_suffix = Bean +# ij_java_session_hi_suffix = Home +# ij_java_session_lhi_prefix = Local +# ij_java_session_lhi_suffix = Home +# ij_java_session_li_prefix = Local +# ij_java_session_si_suffix = Service +# ij_java_space_after_closing_angle_bracket_in_type_argument = false +# ij_java_space_after_colon = true +# ij_java_space_after_comma = true +# ij_java_space_after_comma_in_type_arguments = true +# ij_java_space_after_for_semicolon = true +# ij_java_space_after_quest = true +# ij_java_space_after_type_cast = true +# ij_java_space_before_annotation_array_initializer_left_brace = false +# ij_java_space_before_annotation_parameter_list = false +# ij_java_space_before_array_initializer_left_brace = false +# ij_java_space_before_catch_keyword = true +# ij_java_space_before_catch_left_brace = true +# ij_java_space_before_catch_parentheses = true +# ij_java_space_before_class_left_brace = true +# ij_java_space_before_colon = true +# ij_java_space_before_colon_in_foreach = true +# ij_java_space_before_comma = false +# ij_java_space_before_do_left_brace = true +# ij_java_space_before_else_keyword = true +# ij_java_space_before_else_left_brace = true +# ij_java_space_before_finally_keyword = true +# ij_java_space_before_finally_left_brace = true +# ij_java_space_before_for_left_brace = true +# ij_java_space_before_for_parentheses = true +# ij_java_space_before_for_semicolon = false +# ij_java_space_before_if_left_brace = true +# ij_java_space_before_if_parentheses = true +# ij_java_space_before_method_call_parentheses = false +# ij_java_space_before_method_left_brace = true +# ij_java_space_before_method_parentheses = false +# ij_java_space_before_opening_angle_bracket_in_type_parameter = false +# ij_java_space_before_quest = true +# ij_java_space_before_switch_left_brace = true +# ij_java_space_before_switch_parentheses = true +# ij_java_space_before_synchronized_left_brace = true +# ij_java_space_before_synchronized_parentheses = true +# ij_java_space_before_try_left_brace = true +# ij_java_space_before_try_parentheses = true +# ij_java_space_before_type_parameter_list = false +# ij_java_space_before_while_keyword = true +# ij_java_space_before_while_left_brace = true +# ij_java_space_before_while_parentheses = true +# ij_java_space_inside_one_line_enum_braces = false +# ij_java_space_within_empty_array_initializer_braces = false +# ij_java_space_within_empty_method_call_parentheses = false +# ij_java_space_within_empty_method_parentheses = false +# ij_java_spaces_around_additive_operators = true +# ij_java_spaces_around_assignment_operators = true +# ij_java_spaces_around_bitwise_operators = true +# ij_java_spaces_around_equality_operators = true +# ij_java_spaces_around_lambda_arrow = true +# ij_java_spaces_around_logical_operators = true +# ij_java_spaces_around_method_ref_dbl_colon = false +# ij_java_spaces_around_multiplicative_operators = true +# ij_java_spaces_around_relational_operators = true +# ij_java_spaces_around_shift_operators = true +# ij_java_spaces_around_type_bounds_in_type_parameters = true +# ij_java_spaces_around_unary_operator = false +# ij_java_spaces_within_angle_brackets = false +# ij_java_spaces_within_annotation_parentheses = false +# ij_java_spaces_within_array_initializer_braces = false +# ij_java_spaces_within_braces = false +# ij_java_spaces_within_brackets = false +# ij_java_spaces_within_cast_parentheses = false +# ij_java_spaces_within_catch_parentheses = false +# ij_java_spaces_within_for_parentheses = false +# ij_java_spaces_within_if_parentheses = false +# ij_java_spaces_within_method_call_parentheses = false +# ij_java_spaces_within_method_parentheses = false +# ij_java_spaces_within_parentheses = false +# ij_java_spaces_within_switch_parentheses = false +# ij_java_spaces_within_synchronized_parentheses = false +# ij_java_spaces_within_try_parentheses = false +# ij_java_spaces_within_while_parentheses = false +# ij_java_special_else_if_treatment = true +# ij_java_subclass_name_suffix = Impl +# ij_java_ternary_operation_signs_on_next_line = false +# ij_java_ternary_operation_wrap = off +# ij_java_test_name_suffix = Test +# ij_java_throws_keyword_wrap = off +# ij_java_throws_list_wrap = off +# ij_java_use_external_annotations = false +# ij_java_use_fq_class_names = false +# ij_java_use_relative_indents = false +# ij_java_use_single_class_imports = true +ij_java_variable_annotation_wrap = normal +# ij_java_visibility = public +# ij_java_while_brace_force = never +# ij_java_while_on_new_line = false +# ij_java_wrap_comments = false +ij_java_wrap_first_method_in_call_chain = true +# ij_java_wrap_long_lines = false + +[*.xml] +indent_style = tab +indent_size = 4 + +[*.scala] +indent_style = space +indent_size = 2 + +[*.py] +indent_style = space +indent_size = 4 diff --git a/.gitignore b/.gitignore new file mode 100644 index 00000000..acbe2176 --- /dev/null +++ b/.gitignore @@ -0,0 +1,38 @@ +.eslintcache +.cache +scalastyle-output.xml +.classpath +.idea/* +!.idea/vcs.xml +.metadata +.settings +.project +.version.properties +filter.properties +logs.zip +.mvn/wrapper/*.jar +target +tmp +*.class +*.iml +*.swp +*.jar +*.zip +*.log +*.pyc +.DS_Store +build-target +atlassian-ide-plugin.xml +out/ +/docs/api +/docs/.bundle +/docs/.rubydeps +/docs/ruby2/.bundle +/docs/ruby2/.rubydeps +/docs/.jekyll-metadata +*.ipr +*.iws +tools/flink +tools/flink-* +tools/releasing/release +tools/japicmp-output diff --git a/LICENSE b/LICENSE new file mode 100644 index 00000000..261eeb9e --- /dev/null +++ b/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/NOTICE b/NOTICE new file mode 100644 index 00000000..30367127 --- /dev/null +++ b/NOTICE @@ -0,0 +1,17 @@ +Apache Flink +Copyright 2014-2021 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby +granted, provided that this permission notice appear in all copies. + +THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING +ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, +DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, +WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE +USE OR PERFORMANCE OF THIS SOFTWARE. + + + From 732b388b3cb53a7662871a5fc5e2fa2b9a42d916 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Fri, 1 Apr 2022 12:41:41 +0200 Subject: [PATCH 204/207] [FLINK-26884] refactoring - move Connectors to the root folder --- .../archunit-violations/dd583797-83e1-414c-a38d-330773978813 | 0 .../archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d | 0 .../archunit-violations/stored.rules | 0 .../pom.xml | 0 .../connector/elasticsearch/sink/BulkProcessorBuilderFactory.java | 0 .../flink/connector/elasticsearch/sink/BulkProcessorConfig.java | 0 .../connector/elasticsearch/sink/BulkRequestConsumerFactory.java | 0 .../flink/connector/elasticsearch/sink/ElasticsearchEmitter.java | 0 .../flink/connector/elasticsearch/sink/ElasticsearchSink.java | 0 .../elasticsearch/sink/ElasticsearchSinkBuilderBase.java | 0 .../flink/connector/elasticsearch/sink/ElasticsearchWriter.java | 0 .../flink/connector/elasticsearch/sink/FlushBackoffType.java | 0 .../flink/connector/elasticsearch/sink/NetworkClientConfig.java | 0 .../apache/flink/connector/elasticsearch/sink/RequestIndexer.java | 0 .../connector/elasticsearch/table/AbstractTimeIndexGenerator.java | 0 .../connector/elasticsearch/table/ElasticsearchConfiguration.java | 0 .../elasticsearch/table/ElasticsearchConnectorOptions.java | 0 .../connector/elasticsearch/table/ElasticsearchDynamicSink.java | 0 .../elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java | 0 .../elasticsearch/table/ElasticsearchSinkBuilderSupplier.java | 0 .../elasticsearch/table/ElasticsearchValidationUtils.java | 0 .../flink/connector/elasticsearch/table/IndexGenerator.java | 0 .../flink/connector/elasticsearch/table/IndexGeneratorBase.java | 0 .../connector/elasticsearch/table/IndexGeneratorFactory.java | 0 .../apache/flink/connector/elasticsearch/table/KeyExtractor.java | 0 .../flink/connector/elasticsearch/table/LogicalTypeWithIndex.java | 0 .../connector/elasticsearch/table/RowElasticsearchEmitter.java | 0 .../flink/connector/elasticsearch/table/StaticIndexGenerator.java | 0 .../connectors/elasticsearch/ActionRequestFailureHandler.java | 0 .../connectors/elasticsearch/BufferingNoOpRequestIndexer.java | 0 .../connectors/elasticsearch/ElasticsearchApiCallBridge.java | 0 .../streaming/connectors/elasticsearch/ElasticsearchSinkBase.java | 0 .../connectors/elasticsearch/ElasticsearchSinkFunction.java | 0 .../flink/streaming/connectors/elasticsearch/RequestIndexer.java | 0 .../elasticsearch/table/AbstractTimeIndexGenerator.java | 0 .../elasticsearch/table/ElasticsearchConfiguration.java | 0 .../elasticsearch/table/ElasticsearchConnectorOptions.java | 0 .../elasticsearch/table/ElasticsearchValidationUtils.java | 0 .../streaming/connectors/elasticsearch/table/IndexGenerator.java | 0 .../connectors/elasticsearch/table/IndexGeneratorBase.java | 0 .../connectors/elasticsearch/table/IndexGeneratorFactory.java | 0 .../streaming/connectors/elasticsearch/table/KeyExtractor.java | 0 .../streaming/connectors/elasticsearch/table/RequestFactory.java | 0 .../elasticsearch/table/RowElasticsearchSinkFunction.java | 0 .../connectors/elasticsearch/table/StaticIndexGenerator.java | 0 .../connectors/elasticsearch/util/IgnoringFailureHandler.java | 0 .../connectors/elasticsearch/util/NoOpFailureHandler.java | 0 .../elasticsearch/util/RetryRejectedExecutionFailureHandler.java | 0 .../org/apache/flink/architecture/TestCodeArchitectureTest.java | 0 .../apache/flink/connector/elasticsearch/ElasticsearchUtil.java | 0 .../connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java | 0 .../elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java | 0 .../connector/elasticsearch/sink/ElasticsearchWriterITCase.java | 0 .../apache/flink/connector/elasticsearch/sink/TestClientBase.java | 0 .../apache/flink/connector/elasticsearch/sink/TestEmitter.java | 0 .../elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java | 0 .../table/ElasticsearchDynamicSinkFactoryBaseTest.java | 0 .../flink/connector/elasticsearch/table/IndexGeneratorTest.java | 0 .../flink/connector/elasticsearch/table/KeyExtractorTest.java | 0 .../apache/flink/connector/elasticsearch/table/TestContext.java | 0 .../connectors/elasticsearch/ElasticsearchSinkBaseTest.java | 0 .../connectors/elasticsearch/ElasticsearchSinkTestBase.java | 0 .../elasticsearch/EmbeddedElasticsearchNodeEnvironment.java | 0 .../streaming/connectors/elasticsearch/TestRequestIndexer.java | 0 .../connectors/elasticsearch/table/IndexGeneratorFactoryTest.java | 0 .../connectors/elasticsearch/table/KeyExtractorTest.java | 0 .../streaming/connectors/elasticsearch/table/TestContext.java | 0 .../connectors/elasticsearch/testutils/ElasticsearchResource.java | 0 .../connectors/elasticsearch/testutils/SourceSinkDataTestKit.java | 0 .../src/test/resources/archunit.properties | 0 .../src/test/resources/log4j2-test.properties | 0 .../archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 | 0 .../archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d | 0 .../archunit-violations/stored.rules | 0 .../pom.xml | 0 .../connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java | 0 .../elasticsearch/table/Elasticsearch6Configuration.java | 0 .../elasticsearch/table/Elasticsearch6ConnectorOptions.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkFactory.java | 0 .../elasticsearch/table/Elasticsearch6Configuration.java | 0 .../connectors/elasticsearch/table/Elasticsearch6DynamicSink.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkFactory.java | 0 .../connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java | 0 .../elasticsearch6/Elasticsearch6BulkProcessorIndexer.java | 0 .../streaming/connectors/elasticsearch6/ElasticsearchSink.java | 0 .../streaming/connectors/elasticsearch6/RestClientFactory.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 .../org/apache/flink/architecture/TestCodeArchitectureTest.java | 0 .../elasticsearch/sink/Elasticsearch6SinkBuilderTest.java | 0 .../connector/elasticsearch/sink/Elasticsearch6SinkITCase.java | 0 .../connector/elasticsearch/sink/Elasticsearch6TestClient.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkITCase.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkITCase.java | 0 .../elasticsearch/table/Elasticsearch6DynamicSinkTest.java | 0 .../connectors/elasticsearch6/ElasticsearchSinkITCase.java | 0 .../src/test/resources/archunit.properties | 0 .../src/test/resources/log4j2-test.properties | 0 .../archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f | 0 .../archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e | 0 .../archunit-violations/stored.rules | 0 .../pom.xml | 0 .../connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkFactory.java | 0 .../elasticsearch/table/Elasticsearch7Configuration.java | 0 .../connectors/elasticsearch/table/Elasticsearch7DynamicSink.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkFactory.java | 0 .../connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java | 0 .../elasticsearch7/Elasticsearch7BulkProcessorIndexer.java | 0 .../streaming/connectors/elasticsearch7/ElasticsearchSink.java | 0 .../streaming/connectors/elasticsearch7/RestClientFactory.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 .../org/apache/flink/architecture/TestCodeArchitectureTest.java | 0 .../elasticsearch/sink/Elasticsearch7SinkBuilderTest.java | 0 .../connector/elasticsearch/sink/Elasticsearch7SinkITCase.java | 0 .../connector/elasticsearch/sink/Elasticsearch7TestClient.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkITCase.java | 0 .../elasticsearch/table/Elasticsearch7DynamicSinkTest.java | 0 .../connectors/elasticsearch7/ElasticsearchSinkITCase.java | 0 .../src/test/resources/archunit.properties | 0 .../src/test/resources/log4j2-test.properties | 0 .../pom.xml | 0 .../src/main/resources/META-INF/NOTICE | 0 .../pom.xml | 0 .../src/main/resources/META-INF/NOTICE | 0 129 files changed, 0 insertions(+), 0 deletions(-) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/archunit-violations/dd583797-83e1-414c-a38d-330773978813 (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/archunit-violations/stored.rules (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/pom.xml (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/resources/archunit.properties (100%) rename {flink-connectors/flink-connector-elasticsearch-base => flink-connector-elasticsearch-base}/src/test/resources/log4j2-test.properties (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/archunit-violations/stored.rules (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/pom.xml (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/resources/archunit.properties (100%) rename {flink-connectors/flink-connector-elasticsearch6 => flink-connector-elasticsearch6}/src/test/resources/log4j2-test.properties (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/archunit-violations/stored.rules (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/pom.xml (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/resources/archunit.properties (100%) rename {flink-connectors/flink-connector-elasticsearch7 => flink-connector-elasticsearch7}/src/test/resources/log4j2-test.properties (100%) rename {flink-connectors/flink-sql-connector-elasticsearch6 => flink-sql-connector-elasticsearch6}/pom.xml (100%) rename {flink-connectors/flink-sql-connector-elasticsearch6 => flink-sql-connector-elasticsearch6}/src/main/resources/META-INF/NOTICE (100%) rename {flink-connectors/flink-sql-connector-elasticsearch7 => flink-sql-connector-elasticsearch7}/pom.xml (100%) rename {flink-connectors/flink-sql-connector-elasticsearch7 => flink-sql-connector-elasticsearch7}/src/main/resources/META-INF/NOTICE (100%) diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 b/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 rename to flink-connector-elasticsearch-base/archunit-violations/dd583797-83e1-414c-a38d-330773978813 diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d b/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d rename to flink-connector-elasticsearch-base/archunit-violations/de342dd1-c974-42c9-8f64-ef182ba8c56d diff --git a/flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules b/flink-connector-elasticsearch-base/archunit-violations/stored.rules similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/archunit-violations/stored.rules rename to flink-connector-elasticsearch-base/archunit-violations/stored.rules diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/pom.xml rename to flink-connector-elasticsearch-base/pom.xml diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorBuilderFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkProcessorConfig.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/BulkRequestConsumerFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchEmitter.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSink.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/FlushBackoffType.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/NetworkClientConfig.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/RequestIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/AbstractTimeIndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConfiguration.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchConnectorOptions.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSink.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchSinkBuilderSupplier.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchValidationUtils.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/KeyExtractor.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/LogicalTypeWithIndex.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/RowElasticsearchEmitter.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/StaticIndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ActionRequestFailureHandler.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BufferingNoOpRequestIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/AbstractTimeIndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConfiguration.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchValidationUtils.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractor.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RequestFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/RowElasticsearchSinkFunction.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/StaticIndexGenerator.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/IgnoringFailureHandler.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/NoOpFailureHandler.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java rename to flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/util/RetryRejectedExecutionFailureHandler.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/ElasticsearchUtil.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBaseITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchSinkBuilderBaseTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestClientBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkFactoryBaseTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/KeyExtractorTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/TestContext.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/TestRequestIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/KeyExtractorTest.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/TestContext.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/ElasticsearchResource.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java b/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java rename to flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/testutils/SourceSinkDataTestKit.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties b/flink-connector-elasticsearch-base/src/test/resources/archunit.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/resources/archunit.properties rename to flink-connector-elasticsearch-base/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties rename to flink-connector-elasticsearch-base/src/test/resources/log4j2-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 b/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 rename to flink-connector-elasticsearch6/archunit-violations/25e52d29-fa7e-42fa-a571-b5c76235df52 diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d b/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d rename to flink-connector-elasticsearch6/archunit-violations/db3972e4-f3a3-45b2-9643-27cba0cef09d diff --git a/flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules b/flink-connector-elasticsearch6/archunit-violations/stored.rules similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/archunit-violations/stored.rules rename to flink-connector-elasticsearch6/archunit-violations/stored.rules diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connector-elasticsearch6/pom.xml similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/pom.xml rename to flink-connector-elasticsearch6/pom.xml diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilder.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6Configuration.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6ConnectorOptions.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6Configuration.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6BulkProcessorIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java rename to flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink-connector-elasticsearch6/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkBuilderTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6SinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch6TestClient.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java rename to flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties b/flink-connector-elasticsearch6/src/test/resources/archunit.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/resources/archunit.properties rename to flink-connector-elasticsearch6/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties rename to flink-connector-elasticsearch6/src/test/resources/log4j2-test.properties diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f b/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f rename to flink-connector-elasticsearch7/archunit-violations/1af7baaa-05dc-452a-9de7-653c8b3b324f diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e b/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e rename to flink-connector-elasticsearch7/archunit-violations/e1f30f33-c61c-4707-8c78-a3a80479564e diff --git a/flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules b/flink-connector-elasticsearch7/archunit-violations/stored.rules similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/archunit-violations/stored.rules rename to flink-connector-elasticsearch7/archunit-violations/stored.rules diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connector-elasticsearch7/pom.xml similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/pom.xml rename to flink-connector-elasticsearch7/pom.xml diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilder.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7Configuration.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSink.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7ApiCallBridge.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/Elasticsearch7BulkProcessorIndexer.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSink.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java b/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java rename to flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch7/RestClientFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink-connector-elasticsearch7/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkBuilderTest.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7SinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/sink/Elasticsearch7TestClient.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/connector/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactoryTest.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkTest.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java b/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java rename to flink-connector-elasticsearch7/src/test/java/org/apache/flink/streaming/connectors/elasticsearch7/ElasticsearchSinkITCase.java diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties b/flink-connector-elasticsearch7/src/test/resources/archunit.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/resources/archunit.properties rename to flink-connector-elasticsearch7/src/test/resources/archunit.properties diff --git a/flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties b/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties similarity index 100% rename from flink-connectors/flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties rename to flink-connector-elasticsearch7/src/test/resources/log4j2-test.properties diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-sql-connector-elasticsearch6/pom.xml similarity index 100% rename from flink-connectors/flink-sql-connector-elasticsearch6/pom.xml rename to flink-sql-connector-elasticsearch6/pom.xml diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE similarity index 100% rename from flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE rename to flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-sql-connector-elasticsearch7/pom.xml similarity index 100% rename from flink-connectors/flink-sql-connector-elasticsearch7/pom.xml rename to flink-sql-connector-elasticsearch7/pom.xml diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE similarity index 100% rename from flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE rename to flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE From e948d6eb506693773d41b896002c21169d985cd2 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Fri, 1 Apr 2022 12:45:47 +0200 Subject: [PATCH 205/207] [FLINK-26884] top level pom setup and modification of each pom in submodules. --- flink-connector-elasticsearch-base/pom.xml | 21 +- flink-connector-elasticsearch6/pom.xml | 17 +- flink-connector-elasticsearch7/pom.xml | 16 +- flink-sql-connector-elasticsearch6/pom.xml | 1 - flink-sql-connector-elasticsearch7/pom.xml | 1 - pom.xml | 2012 ++++++++++++++++++++ tools/maven/checkstyle.xml | 562 ++++++ tools/maven/scalastyle-config.xml | 146 ++ tools/maven/spotbugs-exclude.xml | 459 +++++ tools/maven/suppressions.xml | 85 + 10 files changed, 3289 insertions(+), 31 deletions(-) create mode 100644 pom.xml create mode 100644 tools/maven/checkstyle.xml create mode 100644 tools/maven/scalastyle-config.xml create mode 100644 tools/maven/spotbugs-exclude.xml create mode 100644 tools/maven/suppressions.xml diff --git a/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml index 89ae167b..b0f194ac 100644 --- a/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connector-elasticsearch-base/pom.xml @@ -18,8 +18,8 @@ specific language governing permissions and limitations under the License. --> + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 @@ -47,12 +47,12 @@ under the License. org.apache.flink flink-connector-base - ${project.version} + ${flink.version} org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} provided @@ -62,7 +62,7 @@ under the License. org.apache.flink flink-table-api-java-bridge - ${project.version} + ${flink.version} provided true @@ -105,21 +105,20 @@ under the License. org.apache.flink flink-test-utils - ${project.version} test org.apache.flink flink-connector-test-utils - ${project.version} + ${flink.version} test org.apache.flink flink-runtime - ${project.version} + ${flink.version} test-jar test @@ -127,7 +126,7 @@ under the License. org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} test test-jar @@ -136,7 +135,7 @@ under the License. org.apache.flink flink-table-common - ${project.version} + ${flink.version} test-jar test @@ -145,7 +144,7 @@ under the License. org.apache.flink flink-json - ${project.version} + ${flink.version} test diff --git a/flink-connector-elasticsearch6/pom.xml b/flink-connector-elasticsearch6/pom.xml index ad428def..3adae87a 100644 --- a/flink-connector-elasticsearch6/pom.xml +++ b/flink-connector-elasticsearch6/pom.xml @@ -18,8 +18,8 @@ specific language governing permissions and limitations under the License. --> + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd"> 4.0.0 @@ -47,7 +47,7 @@ under the License. org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} provided @@ -57,7 +57,7 @@ under the License. org.apache.flink flink-table-api-java-bridge - ${project.version} + ${flink.version} provided true @@ -102,14 +102,13 @@ under the License. org.apache.flink flink-test-utils - ${project.version} test org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} test test-jar @@ -135,7 +134,7 @@ under the License. org.apache.flink flink-connector-test-utils - ${project.version} + ${flink.version} test @@ -149,7 +148,7 @@ under the License. org.apache.flink flink-table-planner_${scala.binary.version} - ${project.version} + ${flink.version} test @@ -157,7 +156,7 @@ under the License. org.apache.flink flink-json - ${project.version} + ${flink.version} test diff --git a/flink-connector-elasticsearch7/pom.xml b/flink-connector-elasticsearch7/pom.xml index 67348d42..90c8427b 100644 --- a/flink-connector-elasticsearch7/pom.xml +++ b/flink-connector-elasticsearch7/pom.xml @@ -24,10 +24,9 @@ under the License. 4.0.0 - flink-connectors org.apache.flink + flink-connectors 1.16-SNAPSHOT - .. flink-connector-elasticsearch7 @@ -47,7 +46,7 @@ under the License. org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} provided @@ -72,7 +71,7 @@ under the License. org.apache.flink flink-table-api-java-bridge - ${project.version} + ${flink.version} provided true @@ -109,14 +108,13 @@ under the License. org.apache.flink flink-test-utils - ${project.version} test org.apache.flink flink-streaming-java - ${project.version} + ${flink.version} test test-jar @@ -138,7 +136,7 @@ under the License. org.apache.flink flink-connector-test-utils - ${project.version} + ${flink.version} test @@ -146,7 +144,7 @@ under the License. org.apache.flink flink-table-planner_${scala.binary.version} - ${project.version} + ${flink.version} test @@ -154,7 +152,7 @@ under the License. org.apache.flink flink-json - ${project.version} + ${flink.version} test diff --git a/flink-sql-connector-elasticsearch6/pom.xml b/flink-sql-connector-elasticsearch6/pom.xml index bf31dcf1..a3f8651b 100644 --- a/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-sql-connector-elasticsearch6/pom.xml @@ -27,7 +27,6 @@ under the License. org.apache.flink flink-connectors 1.16-SNAPSHOT - .. flink-sql-connector-elasticsearch6 diff --git a/flink-sql-connector-elasticsearch7/pom.xml b/flink-sql-connector-elasticsearch7/pom.xml index a2a5c730..15292f9b 100644 --- a/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-sql-connector-elasticsearch7/pom.xml @@ -27,7 +27,6 @@ under the License. org.apache.flink flink-connectors 1.16-SNAPSHOT - .. flink-sql-connector-elasticsearch7 diff --git a/pom.xml b/pom.xml new file mode 100644 index 00000000..37e7a248 --- /dev/null +++ b/pom.xml @@ -0,0 +1,2012 @@ + + + + + + org.apache + apache + 20 + + + 4.0.0 + + org.apache.flink + flink-connectors + 1.16-SNAPSHOT + Flink : Connectors : + pom + https://flink.apache.org + 2022 + + + + The Apache Software License, Version 2.0 + https://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + https://github.com/apache/flink-connector-elasticsearch + git@github.com:apache/flink-connector-elasticsearch.git + + scm:git:https://gitbox.apache.org/repos/asf/flink-connector-elasticsearch.git + + + + + flink-connector-elasticsearch-base + flink-connector-elasticsearch6 + flink-connector-elasticsearch7 + + + + UTF-8 + UTF-8 + + + 1C + + ${flink.forkCount} + true + + 1.15-SNAPSHOT + 15.0 + 2.12.4 + 1.8 + 1.7.32 + 2.17.1 + + ${target.java.version} + ${target.java.version} + 2.1.1 + + 2.12.7 + 2.12 + 0.7.6 + + 4.13.2 + 5.8.1 + 3.21.0 + 0.22.0 + 1.16.2 + + 2.21.0 + 2.0.9 + 1.3 + + 0.10.9.3 + 2.27.0 + 3.17.3 + 3.14.9 + 1.8.0 + + false + 1.14.0 + tools/japicmp-output + + validate + + 2.4.2 + + + + **/*Test.* + + + + + org.apache.flink + flink-shaded-force-shading + ${flink.shaded.version} + + + + + + + org.slf4j + slf4j-api + provided + + + + + com.google.code.findbugs + jsr305 + provided + + + + + org.junit.jupiter + junit-jupiter + test + + + + org.junit.vintage + junit-vintage-engine + test + + + + org.assertj + assertj-core + test + + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + + + org.powermock + powermock-module-junit4 + ${powermock.version} + jar + test + + + + org.powermock + powermock-api-mockito2 + ${powermock.version} + jar + test + + + org.mockito + mockito-core + + + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + jar + test + + + + org.testcontainers + junit-jupiter + test + + + + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + org.apache.logging.log4j + log4j-api + test + + + + org.apache.logging.log4j + log4j-core + test + + + + + org.apache.logging.log4j + log4j-1.2-api + test + + + + org.apache.flink + flink-test-utils-junit + test + + + + + + + + + + + + + + org.apache.flink + flink-test-utils + ${flink.version} + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + org.apache.flink + flink-test-utils-junit + ${flink.version} + test + + + + + org.apache.flink + flink-architecture-tests-base + ${flink.version} + test + + + + org.apache.flink + flink-architecture-tests-test + ${flink.version} + test + + + + + com.google.code.findbugs + jsr305 + 1.3.9 + + + + org.slf4j + slf4j-api + ${slf4j.version} + + + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + + + + org.apache.logging.log4j + log4j-api + ${log4j.version} + + + + org.apache.logging.log4j + log4j-core + ${log4j.version} + + + + + org.apache.logging.log4j + log4j-1.2-api + ${log4j.version} + + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + org.xerial.snappy + snappy-java + 1.1.8.3 + + + + org.lz4 + lz4-java + ${lz4.version} + + + + com.github.oshi + oshi-core + 3.4.0 + + + + + org.apache.avro + avro + ${avro.version} + + + + + org.hamcrest + hamcrest-core + ${hamcrest.version} + + + + + net.bytebuddy + byte-buddy + 1.8.22 + + + + + net.bytebuddy + byte-buddy-agent + 1.8.22 + + + + + org.objenesis + objenesis + 2.1 + + + + + commons-logging + commons-logging + 1.1.3 + + + + com.fasterxml.jackson + jackson-bom + pom + import + 2.13.0 + + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + + com.squareup.okhttp3 + logging-interceptor + ${okhttp.version} + + + + + jakarta.activation + jakarta.activation-api + 1.2.1 + provided + + + + + jakarta.xml.bind + jakarta.xml.bind-api + 2.3.2 + provided + + + + + org.junit + junit-bom + ${junit5.version} + pom + import + + + + junit + junit + ${junit4.version} + + + + org.assertj + assertj-core + ${assertj.version} + test + + + + + commons-cli + commons-cli + 1.5.0 + + + + commons-io + commons-io + 2.11.0 + + + + + commons-collections + commons-collections + 3.2.2 + + + + + commons-configuration + commons-configuration + 1.7 + + + + commons-codec + commons-codec + 1.15 + + + + org.apache.commons + commons-math3 + 3.6.1 + + + + org.apache.commons + commons-compress + 1.21 + + + + + org.javassist + javassist + 3.24.0-GA + + + + + joda-time + joda-time + 2.5 + + + + org.joda + joda-convert + 1.7 + + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + org.scala-lang + scala-library + ${scala.version} + + + + org.scala-lang + scala-reflect + ${scala.version} + + + + org.scala-lang + scala-compiler + ${scala.version} + + + + org.scalatest + scalatest_${scala.binary.version} + 3.0.0 + test + + + + + javax.xml.bind + jaxb-api + ${jaxb.api.version} + + + + + javax.activation + javax.activation-api + ${javax.activation.api.version} + + + + + org.apache.httpcomponents + httpcore + 4.4.14 + + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + + org.reflections + reflections + 0.9.10 + test + + + + + com.lmax + disruptor + 3.4.2 + + + + org.yaml + snakeyaml + 1.27 + + + io.netty + netty-bom + 4.1.70.Final + pom + import + + + org.testcontainers + testcontainers-bom + ${testcontainers.version} + pom + import + + + + com.tngtech.archunit + archunit + ${archunit.version} + test + + + + com.tngtech.archunit + archunit-junit5 + ${archunit.version} + test + + + + + + + + + + + sql-jars + + + !skipSqlJars + + + + flink-sql-connector-elasticsearch6 + flink-sql-connector-elasticsearch7 + + + + + scala-2.12 + + 2.12.7 + 2.12 + + + + !old-and-unsupported-scala-version + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-versions + + enforce + + + + + + *:*_2.11 + *:*_2.10 + + Scala 2.10/2.11 dependencies are not allowed for Scala 2.12 builds. This can be caused by hard-coded scala versions, where the 'scala.binary.version' property should be used instead. + + + + + + + + + + + + enable-adaptive-scheduler + + + + org.apache.maven.plugins + maven-surefire-plugin + + + true + + + + + + + + + java11 + + [11,) + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.7 + + + org.apache.maven.plugins + maven-shade-plugin + 3.2.4 + + + com.github.siom79.japicmp + japicmp-maven-plugin + + + javax.xml.bind + jaxb-api + 2.3.0 + + + com.sun.xml.bind + jaxb-impl + 2.3.1 + + + com.sun.xml.bind + jaxb-core + 2.3.0 + + + javax.activation + activation + 1.1.1 + + + + + org.apache.maven.plugins + maven-surefire-plugin + + org.apache.flink.testutils.junit.FailsOnJava11 + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + + + + + + + + + + java17 + + [17,) + + + + + + + com.diffplug.spotless + spotless-maven-plugin + + + true + + + + + + + + + java11-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 11 + 11 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + java17-target + + + + org.apache.maven.plugins + maven-compiler-plugin + + 17 + 17 + + --add-exports=java.base/sun.net.util=ALL-UNNAMED + --add-exports=java.management/sun.management=ALL-UNNAMED + --add-exports=java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports=java.security.jgss/sun.security.krb5=ALL-UNNAMED + + + + + + + + + fast + + + fast + + + + + + + org.apache.rat + apache-rat-plugin + + true + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + true + + + + com.diffplug.spotless + spotless-maven-plugin + + true + + + + org.scalastyle + scalastyle-maven-plugin + + true + + + + org.apache.maven.plugins + maven-enforcer-plugin + + true + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + com.github.siom79.japicmp + japicmp-maven-plugin + + true + + + + + + + + + check-convergence + + + check-convergence + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + dependency-convergence + ${flink.convergence.phase} + + + + + + + + + spotbugs + + + spotbugs + + + + + + com.github.hazendaz.spotbugs + spotbugs-maven-plugin + 3.0.6 + + + + findbugs-run + compile + + check + + + + + + true + Low + default + ${project.build.directory}/spotbugs + ${rootDir}/tools/maven/spotbugs-exclude.xml + true + + + + + org.codehaus.mojo + xml-maven-plugin + 1.0.1 + + + verify + + transform + + + + + + + ${project.build.directory}/spotbugs + ${project.build.directory}/spotbugs + + plain.xsl + + + + .html + + + + + + + + com.github.hazendaz.spotbugs + spotbugs-maven-plugin + 3.0.6 + + + + + + + + + + aggregate-scaladoc + + + + + + docs-and-source + + + docs-and-source + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.2.1 + + + attach-sources + + jar + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + true + + + + attach-javadocs + + jar + + + + + + + + + + release + + + release + + + + 1.8 + + + + + org.apache.maven.plugins + maven-gpg-plugin + 1.4 + + + sign-artifacts + verify + + sign + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + (,3.3) + + + 1.8.0 + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + attach-javadocs + + jar + + + + + + + + + org.apache.maven.plugins + maven-release-plugin + 2.1 + + forked-path + false + ${arguments} -Psonatype-oss-release + + + + + + + + + + + + + org.apache.felix + maven-bundle-plugin + 3.0.1 + true + true + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + false + + true + true + + + + + + + org.apache.rat + apache-rat-plugin + 0.13 + false + + + verify + + check + + + + + true + false + 0 + + + + AL2 + Apache License 2.0 + + + Licensed to the Apache Software Foundation (ASF) under one + + + + + + Apache License 2.0 + + + + + **/.*/** + **/*.prefs + **/*.log + + docs/**/jquery* + docs/**/bootstrap* + docs/themes/book/** + docs/**/anchor* + **/resources/**/font-awesome/** + **/resources/**/jquery* + **/resources/**/bootstrap* + docs/resources/** + docs/public/** + docs/assets/github.css + docs/static/flink-header-logo.svg + docs/static/figs/*.svg + docs/static/font-awesome/** + docs/static/flink-header-logo.svg + docs/static/figs/*.svg + flink-clients/src/main/resources/web-docs/js/*d3.js + + + **/packaged_licenses/LICENSE.*.txt + **/licenses/LICENSE* + **/licenses-binary/LICENSE* + + + flink-runtime-web/web-dashboard/package.json + flink-runtime-web/web-dashboard/package-lock.json + flink-runtime-web/web-dashboard/angular.json + flink-runtime-web/web-dashboard/proxy.conf.json + flink-runtime-web/web-dashboard/tsconfig.json + flink-runtime-web/web-dashboard/tslint.json + flink-runtime-web/web-dashboard/src/browserslist + flink-runtime-web/web-dashboard/src/tsconfig.app.json + flink-runtime-web/web-dashboard/src/tsconfig.spec.json + flink-runtime-web/web-dashboard/src/tslint.json + + + flink-runtime-web/web-dashboard/src/assets/** + + + flink-runtime-web/web-dashboard/web/** + + + flink-runtime-web/web-dashboard/node_modules/** + flink-runtime-web/web-dashboard/node/** + + + flink-table/flink-table-code-splitter/src/main/antlr4/** + + + **/src/test/resources/*-data + flink-tests/src/test/resources/testdata/terainput.txt + flink-formats/flink-avro/src/test/resources/flink_11-kryo_registrations + flink-scala/src/test/resources/flink_11-kryo_registrations + flink-core/src/test/resources/kryo-serializer-config-snapshot-v1 + flink-formats/flink-avro/src/test/resources/avro/*.avsc + out/test/flink-avro/avro/user.avsc + flink-table/flink-sql-client/src/test/resources/*.out + flink-table/flink-table-planner/src/test/resources/**/*.out + flink-table/flink-table-planner/src/test/resources/json/*.json + flink-yarn/src/test/resources/krb5.keytab + flink-end-to-end-tests/test-scripts/test-data/** + flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/keystore.jks + flink-connectors/flink-connector-kafka/src/test/resources/** + flink-connectors/flink-connector-hive/src/test/resources/** + flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/answer_set/* + flink-end-to-end-tests/flink-tpcds-test/tpcds-tool/query/* + flink-connectors/flink-connector-aws-base/src/test/resources/profile + flink-connectors/flink-connector-kinesis/src/test/resources/profile + flink-connectors/flink-connector-aws-kinesis-streams/src/test/resources/profile + flink-table/flink-table-code-splitter/src/test/resources/** + flink-connectors/flink-connector-pulsar/src/test/resources/** + + + **/archunit-violations/** + + + **/src/test/resources/serializer-snapshot-* + **/src/test/resources/**/serializer-snapshot + **/src/test/resources/**/test-data + **/src/test/resources/*-snapshot + **/src/test/resources/*.snapshot + **/src/test/resources/*-savepoint/** + **/src/test/resources/*-savepoint-native/** + **/src/test/resources/*-checkpoint/** + flink-core/src/test/resources/serialized-kryo-serializer-1.3 + flink-core/src/test/resources/type-without-avro-serialized-using-kryo + flink-formats/flink-avro/src/test/resources/flink-1.4-serializer-java-serialized + + flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/avro/generated/* + flink-end-to-end-tests/flink-state-evolution-test/savepoints/* + flink-formats/flink-avro/src/test/resources/testdata.avro + flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/generated/*.java + flink-formats/flink-avro-confluent-registry/src/test/resources/*.json + flink-formats/flink-avro-confluent-registry/src/test/resources/*.avro + flink-formats/flink-json/src/test/resources/*.txt + flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/generated/*.java + flink-formats/flink-parquet/src/test/resources/avro/** + flink-formats/flink-parquet/src/test/resources/protobuf/** + + flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/AbstractByteBufTest.java + + **/flink-bin/conf/workers + **/flink-bin/conf/masters + + **/README.md + .github/** + + **/*.iml + flink-quickstart/**/testArtifact/goal.txt + + out/** + **/target/** + **/scalastyle-output.xml + build-target/** + docs/layouts/shortcodes/generated/** + docs/static/generated/** + + tools/artifacts/** + tools/flink*/** + + tools/japicmp-output/** + + tools/releasing/release/** + + apache-maven-3.2.5/** + + **/.idea/** + + flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/example/avro/** + flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/avro/** + + flink-jepsen/store/** + flink-jepsen/docker/id_rsa* + flink-jepsen/docker/nodes + + + flink-python/lib/** + flink-python/dev/download/** + flink-python/docs/_build/** + + + **/awssdk/global/handlers/execution.interceptors + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.0.0-M5 + + + + ${flink.forkCount} + ${flink.reuseForks} + false + + 0${surefire.forkNumber} + true + true + + random + ${project.basedir} + + ${test.randomization.seed} + true + + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC -Duser.country=US -Duser.language=en + + + + + default-test + test + + test + + + + ${test.unit.pattern} + + + + + + integration-tests + integration-test + + test + + + + **/*.* + + + ${test.unit.pattern} + + **/*$* + + false + + + + + + org.apache.maven.plugins + maven-eclipse-plugin + 2.8 + + + + org.eclipse.jdt.launching.JRE_CONTAINER + + + true + true + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce-maven + + enforce + + + + + + [3.1.1,) + + + ${target.java.version} + + + + + + ban-unsafe-snakeyaml + + enforce + + + + + + org.yaml:snakeyaml:(,1.26] + + + + org.yaml:snakeyaml:(,1.26]:*:test + + Older snakeyaml versions are not allowed due to security vulnerabilities. + + + + + + ban-unsafe-jackson + + enforce + + + + + + com.fasterxml.jackson*:*:(,2.12.0] + + Older jackson versions are not allowed due to security vulnerabilities. + + + + + + forbid-log4j-1 + + enforce + + + + + + log4j:log4j + org.slf4j:slf4j-log4j12 + + Log4j 1 dependencies are not allowed because they conflict with Log4j 2. If the dependency absolutely requires the Log4j 1 API, use 'org.apache.logging.log4j:log4j-1.2-api'. + + + + + + forbid-direct-akka-rpc-dependencies + + enforce + + + + + + org.apache.flink:flink-rpc-akka + + + Direct dependencies on flink-rpc-akka are not allowed. Depend on flink-rpc-akka-loader instead, and use RpcSystem#load or the TestingRpcService. + + + + + + + forbid-direct-table-planner-dependencies + + enforce + + + + + + org.apache.flink:flink-table-planner_${scala.binary.version} + + + org.apache.flink:flink-table-planner_${scala.binary.version}:*:*:test + + + Direct dependencies on flink-table-planner are not allowed. + You should depend on either Table API modules or flink-table-planner-loader. + + + + + + + dependency-convergence + + none + + enforce + + + + + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + + + * + + + log4j.properties + log4j2.properties + log4j-test.properties + log4j2-test.properties + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + **/META-INF/maven/?*/?*/** + + + + + + + + + Apache Flink + UTF-8 + + + + + + shade-flink + package + + shade + + + false + false + true + ${project.basedir}/target/dependency-reduced-pom.xml + + + + + org.apache.flink:flink-shaded-force-shading + + ** + + + + + io.netty:netty + + META-INF/LICENSE.txt + + + + + + + org.apache.flink:flink-shaded-force-shading + + + + + + + + + + org.commonjava.maven.plugins + directory-maven-plugin + 0.1 + + + directories + + directory-of + + initialize + + rootDir + + org.apache.flink + flink-connectors + + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.8.0 + + ${target.java.version} + ${target.java.version} + + false + + + -Xpkginfo:always + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.17 + + + com.puppycrawl.tools + checkstyle + + 8.14 + + + + + validate + validate + + check + + + + + /tools/maven/suppressions.xml + true + /tools/maven/checkstyle.xml + true + true + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + 1.7 + + + + + + org.apache.flink,org.apache.flink.shaded,,javax,java,scala,\# + + + + + + + + spotless-check + validate + + check + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.9.1 + + true + false + + -Xdoclint:none + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.0.0-M1 + + + + org.apache.maven.plugins + maven-dependency-plugin + 3.2.0 + + + + org.apache.flink:* + + org.hamcrest:hamcrest-core + + org.powermock:powermock-core + org.powermock:powermock-reflect + org.powermock:powermock-api-support + + + + org.apache.flink:force-shading + + com.google.code.findbugs:jsr305 + org.scala-lang:scala-compiler + + org.slf4j:slf4j-api + + log4j:log4j + org.slf4j:slf4j-log4j12 + + org.apache.logging.log4j:log4j-slf4j-impl + org.apache.logging.log4j:log4j-api + org.apache.logging.log4j:log4j-core + org.apache.logging.log4j:log4j-1.2-api + + org.apache.flink:flink-test-utils-junit + junit:junit + org.mockito:mockito-core + org.powermock:powermock-api-mockito2 + org.powermock:powermock-module-junit4 + org.hamcrest:hamcrest-all + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + + + org.apache.maven.plugins + maven-site-plugin + + + attach-descriptor + none + + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + + validate + + check + + + + + false + true + true + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + ${project.basedir}/target/scalastyle-output.xml + UTF-8 + UTF-8 + + + + + + net.alchim31.maven + scala-maven-plugin + 3.2.2 + + + -nobootcp + -target:jvm-${target.java.version} + + + -Xss2m + + + + + + + com.github.siom79.japicmp + japicmp-maven-plugin + 0.11.0 + + + + org.apache.flink + ${project.artifactId} + ${japicmp.referenceVersion} + ${project.packaging} + + + + + ${project.build.directory}/${project.artifactId}-${project.version}.${project.packaging} + + + + true + + @org.apache.flink.annotation.Public + + + + + @org.apache.flink.annotation.Experimental + @org.apache.flink.annotation.PublicEvolving + @org.apache.flink.annotation.Internal + org.apache.flink.streaming.api.datastream.DataStream#DataStream(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment,org.apache.flink.streaming.api.transformations.StreamTransformation) + org.apache.flink.streaming.api.environment.LegacyLocalStreamEnvironment + org.apache.flink.streaming.api.functions.sink.RichSinkFunction#invoke(java.lang.Object) + org.apache.flink.streaming.api.functions.sink.SinkFunction + org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapred.HadoopOutputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopInputFormat + org.apache.flink.api.scala.hadoop.mapreduce.HadoopOutputFormat + + public + false + true + true + false + true + false + true + + true + + ${rootDir}/${japicmp.outputDir}/${project.artifactId} + + + org.apache.flink + flink-annotations + ${project.version} + + + + + + verify + + cmp + + + + + + + + org.owasp + dependency-check-maven + 5.0.0-M2 + + ALL + true + true + + *flink-docs + *flink-end-to-end-tests + *flink-fs-tests* + *flink-yarn-tests* + + + + + + + diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml new file mode 100644 index 00000000..2048fd18 --- /dev/null +++ b/tools/maven/checkstyle.xml @@ -0,0 +1,562 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml new file mode 100644 index 00000000..53b10529 --- /dev/null +++ b/tools/maven/scalastyle-config.xml @@ -0,0 +1,146 @@ + + + + + + + + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/spotbugs-exclude.xml b/tools/maven/spotbugs-exclude.xml new file mode 100644 index 00000000..b165464b --- /dev/null +++ b/tools/maven/spotbugs-exclude.xml @@ -0,0 +1,459 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + D + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml new file mode 100644 index 00000000..129a0c7a --- /dev/null +++ b/tools/maven/suppressions.xml @@ -0,0 +1,85 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + From 7bfc33d4d95702a90d36de05100d1867cbaa3c91 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Fri, 1 Apr 2022 12:47:37 +0200 Subject: [PATCH 206/207] [FLINK-26884] fix dependency convergence issue --- flink-connector-elasticsearch-base/pom.xml | 6 ++++++ pom.xml | 13 +++++++++++++ 2 files changed, 19 insertions(+) diff --git a/flink-connector-elasticsearch-base/pom.xml b/flink-connector-elasticsearch-base/pom.xml index b0f194ac..a5afd009 100644 --- a/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connector-elasticsearch-base/pom.xml @@ -92,6 +92,12 @@ under the License. org.elasticsearch.client elasticsearch-rest-high-level-client ${elasticsearch.version} + + + org.apache.httpcomponents + httpcore-nio + + diff --git a/pom.xml b/pom.xml index 37e7a248..01587eeb 100644 --- a/pom.xml +++ b/pom.xml @@ -236,6 +236,11 @@ under the License. test + + org.apache.httpcomponents + httpcore-nio + + @@ -587,6 +592,14 @@ under the License. 4.5.13 + + + org.apache.httpcomponents + httpcore-nio + 4.4.12 + + org.reflections reflections From 60464b1bc6c118a0e299fcddbd211aa8fc2bafc3 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Sun, 3 Apr 2022 21:38:53 +0200 Subject: [PATCH 207/207] [FLINK-26884] add @deprecated --- .../elasticsearch/table/ElasticsearchConnectorOptions.java | 7 ++++++- .../elasticsearch/table/Elasticsearch6DynamicSink.java | 5 +++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 38d65a7f..4838b035 100644 --- a/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -30,7 +30,12 @@ import static org.apache.flink.configuration.description.TextElement.text; -/** Options for the Elasticsearch connector. */ +/** + * Options for the Elasticsearch connector. + * + * @deprecated This has been deprecated and will be removed in the future. + */ +@Deprecated @PublicEvolving public class ElasticsearchConnectorOptions { diff --git a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java index 0c7c756d..1a2cdd18 100644 --- a/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java +++ b/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSink.java @@ -53,7 +53,12 @@ /** * A {@link DynamicTableSink} that describes how to create a {@link ElasticsearchSink} from a * logical description. + * + * @deprecated Please use {@link + * org.apache.flink.connector.elasticsearch.sink.Elasticsearch6SinkBuilder} to build a {@link + * org.apache.flink.connector.elasticsearch.sink.ElasticsearchSink} instead. */ +@Deprecated @PublicEvolving final class Elasticsearch6DynamicSink implements DynamicTableSink { @VisibleForTesting