-The difference is that we now need to provide a list of Elasticsearch Nodes
-to which the sink should connect using a `TransportClient`.
+The difference is that now we do not need to provide a list of addresses
+of Elasticsearch nodes.
More information about Elasticsearch can be found [here](https://elastic.co).
+
+#### Packaging the Elasticsearch Connector into an Uber-Jar
+
+For the execution of your Flink program, it is recommended to build a
+so-called uber-jar (executable jar) containing all your dependencies
+(see [here]({{site.baseurl}}/dev/linking.html) for further information).
+
+However, when an uber-jar containing an Elasticsearch sink is executed,
+an `IllegalArgumentException` may occur, which is caused by conflicting
+files of Elasticsearch and it's dependencies in `META-INF/services`:
+
+```
+IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist. You need to add the corresponding JAR file supporting this SPI to your classpath. The current classpath supports the following names: [es090, completion090, XBloomFilter]]
+```
+
+If the uber-jar is built using Maven, this issue can be avoided by
+adding the following to the Maven POM file in the plugins section:
+
+~~~xml
+
+~~~
diff --git a/docs/dev/connectors/elasticsearch2.md b/docs/dev/connectors/elasticsearch2.md
deleted file mode 100644
index af02c84c4ab90..0000000000000
--- a/docs/dev/connectors/elasticsearch2.md
+++ /dev/null
@@ -1,173 +0,0 @@
----
-title: "Elasticsearch 2.x Connector"
-nav-title: Elasticsearch 2.x
-nav-parent_id: connectors
-nav-pos: 5
----
-
-
-This connector provides a Sink that can write to an
-[Elasticsearch 2.x](https://elastic.co/) Index. To use this connector, add the
-following dependency to your project:
-
-{% highlight xml %}
-
-{% endhighlight %}
-
-Note that the streaming connectors are currently not part of the binary
-distribution. See
-[here]({{site.baseurl}}/dev/linking.html)
-for information about how to package the program with the libraries for
-cluster execution.
-
-#### Installing Elasticsearch 2.x
-
-Instructions for setting up an Elasticsearch cluster can be found
-[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/setup.html).
-Make sure to set and remember a cluster name. This must be set when
-creating a Sink for writing to your cluster
-
-#### Elasticsearch 2.x Sink
-The connector provides a Sink that can send data to an Elasticsearch 2.x Index.
-
-The sink communicates with Elasticsearch via Transport Client
-
-See [here](https://www.elastic.co/guide/en/elasticsearch/client/java-api/current/transport-client.html)
-for information about the Transport Client.
-
-The code below shows how to create a sink that uses a `TransportClient` for communication:
-
-
-
-A Map of Strings is used to configure the Sink. The configuration keys
-are documented in the Elasticsearch documentation
-[here](https://www.elastic.co/guide/en/elasticsearch/reference/current/index.html).
-Especially important is the `cluster.name`. parameter that must correspond to
-the name of your cluster and with ElasticSearch 2x you also need to specify `path.home`.
-
-Internally, the sink uses a `BulkProcessor` to send Action requests to the cluster.
-This will buffer elements and Action Requests before sending to the cluster. The behaviour of the
-`BulkProcessor` can be configured using these config keys:
- * **bulk.flush.max.actions**: Maximum amount of elements to buffer
- * **bulk.flush.max.size.mb**: Maximum amount of data (in megabytes) to buffer
- * **bulk.flush.interval.ms**: Interval at which to flush data regardless of the other two
- settings in milliseconds
-
-This now provides a list of Elasticsearch Nodes
-to which the sink should connect via a `TransportClient`.
-
-More information about Elasticsearch can be found [here](https://elastic.co).
-
-
-#### Packaging the Elasticsearch Connector into an Uber-jar
-
-For the execution of your Flink program,
-it is recommended to build a so-called uber-jar (executable jar) containing all your dependencies
-(see [here]({{site.baseurl}}/dev/linking.html) for further information).
-
-However,
-when an uber-jar containing an Elasticsearch sink is executed,
-an `IllegalArgumentException` may occur,
-which is caused by conflicting files of Elasticsearch and it's dependencies
-in `META-INF/services`:
-
-```
-IllegalArgumentException[An SPI class of type org.apache.lucene.codecs.PostingsFormat with name 'Lucene50' does not exist. You need to add the corresponding JAR file supporting this SPI to your classpath. The current classpath supports the following names: [es090, completion090, XBloomFilter]]
-```
-
-If the uber-jar is build by means of maven,
-this issue can be avoided by adding the following bits to the pom file:
-
-```
-
-```
diff --git a/docs/dev/connectors/elasticsearch5.md b/docs/dev/connectors/elasticsearch5.md
deleted file mode 100644
index 2673d860b9179..0000000000000
--- a/docs/dev/connectors/elasticsearch5.md
+++ /dev/null
@@ -1,146 +0,0 @@
----
-title: "Elasticsearch 5.x Connector"
-nav-title: Elasticsearch 5.x
-nav-parent_id: connectors
-nav-pos: 6
----
-
-
-This connector provides a Sink that can write to an
-[Elasticsearch 5.x](https://elastic.co/) Index. To use this connector, add the
-following dependency to your project:
-
-{% highlight xml %}
-
-{% endhighlight %}
-
-Note that the streaming connectors are currently not part of the binary
-distribution. See
-[here]({{site.baseurl}}/dev/cluster_execution.html#linking-with-modules-not-contained-in-the-binary-distribution)
-for information about how to package the program with the libraries for
-cluster execution.
-
-#### Installing Elasticsearch 5.x
-
-Instructions for setting up an Elasticsearch cluster can be found
- [here](https://www.elastic.co/guide/en/elasticsearch/reference/5.x/setup.html).
-Make sure to set and remember a cluster name. This must be set when
-creating a Sink for writing to your cluster
-
-#### Elasticsearch 5.x Sink
-The connector provides a Sink that can send data to an Elasticsearch 5.x Index.
-
-The sink communicates with Elasticsearch via Transport Client
-
-See [here](https://www.elastic.co/guide/en/elasticsearch/client/java-api/5.x/transport-client.html)
-for information about the Transport Client.
-
-The code below shows how to create a sink that uses a `TransportClient` for communication:
-
-
-
-The first Map of Strings is used to configure the Transport Client. The configuration keys
-are documented in the Elasticsearch documentation
-[here](https://www.elastic.co/guide/en/elasticsearch/reference/5.x/index.html).
-Especially important is the `cluster.name`. parameter that must correspond to
-the name of your cluster.
-
-The second Map of Strings is used to configure a `BulkProcessor` to send Action requests to the cluster.
-This will buffer elements and Action Requests before sending to the cluster. The behaviour of the
-`BulkProcessor` can be configured using these config keys:
- * **bulk.flush.max.actions**: Maximum amount of elements to buffer
- * **bulk.flush.max.size.mb**: Maximum amount of data (in megabytes) to buffer
- * **bulk.flush.interval.ms**: Interval at which to flush data regardless of the other two
- settings in milliseconds
-
-This now provides a list of Elasticsearch Nodes
-to which the sink should connect via a `TransportClient`.
-
-More information about Elasticsearch can be found [here](https://elastic.co).
-
diff --git a/docs/dev/connectors/filesystem_sink.md b/docs/dev/connectors/filesystem_sink.md
index bcaeb17728e2b..67250f0c83324 100644
--- a/docs/dev/connectors/filesystem_sink.md
+++ b/docs/dev/connectors/filesystem_sink.md
@@ -2,7 +2,7 @@
title: "HDFS Connector"
nav-title: Rolling File Sink
nav-parent_id: connectors
-nav-pos: 7
+nav-pos: 5
---
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 0000000000000..81652c45758c6
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml
@@ -0,0 +1,95 @@
+
+
+
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 0000000000000..d8025501adbf6
--- /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 0000000000000..6298a857d8df1
--- /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 0000000000000..6a2d65fee5650
--- /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-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java
similarity index 71%
rename from flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkFunction.java
rename to flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java
index 752a83e9a7f0e..1e20a0a30093c 100644
--- a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkFunction.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkFunction.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -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,
@@ -15,18 +15,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.flink.streaming.connectors.elasticsearch5;
+
+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;
/**
- * Method that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
+ * Creates multiple {@link ActionRequest ActionRequests} from an element in a stream.
*
*
- * This is used by {@link ElasticsearchSink} to prepare elements for sending them to Elasticsearch.
+ * This is used by sinks to prepare elements for sending them to Elasticsearch.
*
*
* Example:
@@ -56,5 +58,14 @@
* @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 0000000000000..4587a80797057
--- /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 0000000000000..9776c4c9eb6ff
--- /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 0000000000000..2f9e4c172415e
--- /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 0000000000000..f59eb03c5f823
--- /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 0000000000000..55a48fae94056
--- /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-elasticsearch5/src/test/resources/log4j2.properties b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties
similarity index 93%
rename from flink-connectors/flink-connector-elasticsearch5/src/test/resources/log4j2.properties
rename to flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties
index dc207262c0760..20551848eea24 100644
--- a/flink-connectors/flink-connector-elasticsearch5/src/test/resources/log4j2.properties
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/resources/log4j-test.properties
@@ -16,12 +16,12 @@
# limitations under the License.
################################################################################
-log4j.rootLogger=OFF, testlogger
+log4j.rootLogger=INFO, testlogger
log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
+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
\ No newline at end of file
+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 0000000000000..45b3b92f0cf65
--- /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
diff --git a/flink-connectors/flink-connector-elasticsearch/pom.xml b/flink-connectors/flink-connector-elasticsearch/pom.xml
index b2d7284d78cb1..14f28d0add6a1 100644
--- a/flink-connectors/flink-connector-elasticsearch/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch/pom.xml
@@ -52,9 +52,9 @@ under the License.
- org.elasticsearch
- elasticsearch
- ${elasticsearch.version}
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
@@ -73,18 +73,15 @@ under the License.
test
test-jar
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
- 3
-
-
-
-
+
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
+ test
+ test-jar
+
+
+
diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/Elasticsearch1ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/Elasticsearch1ApiCallBridge.java
new file mode 100644
index 0000000000000..098afa9c9c329
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/Elasticsearch1ApiCallBridge.java
@@ -0,0 +1,128 @@
+/*
+ * 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.util.Preconditions;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.node.Node;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
+import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+
+/**
+ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 1.x.
+ */
+public class Elasticsearch1ApiCallBridge implements ElasticsearchApiCallBridge {
+
+ private static final long serialVersionUID = -2632363720584123682L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch1ApiCallBridge.class);
+
+ /** User-provided transport addresses. This is null if we are using an embedded {@link Node} for communication. */
+ private final List transportAddresses;
+
+ /** The embedded {@link Node} used for communication. This is null if we are using a TransportClient. */
+ private transient Node node;
+
+ /**
+ * Constructor for use of an embedded {@link Node} for communication with the Elasticsearch cluster.
+ */
+ Elasticsearch1ApiCallBridge() {
+ this.transportAddresses = null;
+ }
+
+ /**
+ * Constructor for use of a {@link TransportClient} for communication with the Elasticsearch cluster.
+ */
+ Elasticsearch1ApiCallBridge(List transportAddresses) {
+ Preconditions.checkArgument(transportAddresses != null && !transportAddresses.isEmpty());
+ this.transportAddresses = transportAddresses;
+ }
+
+ @Override
+ public Client createClient(Map clientConfig) {
+ if (transportAddresses == null) {
+
+ // Make sure that we disable http access to our embedded node
+ Settings settings = settingsBuilder()
+ .put(clientConfig)
+ .put("http.enabled", false)
+ .build();
+
+ node = nodeBuilder()
+ .settings(settings)
+ .client(true)
+ .data(false)
+ .node();
+
+ Client client = node.client();
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Created Elasticsearch client from embedded node");
+ }
+
+ return client;
+ } else {
+ Settings settings = settingsBuilder()
+ .put(clientConfig)
+ .build();
+
+ TransportClient transportClient = new TransportClient(settings);
+ for (TransportAddress transport: transportAddresses) {
+ transportClient.addTransportAddress(transport);
+ }
+
+ // verify that we actually are connected to a cluster
+ if (transportClient.connectedNodes().isEmpty()) {
+ throw new RuntimeException("Elasticsearch client is not connected to any Elasticsearch nodes!");
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Created Elasticsearch TransportClient with connected nodes {}", transportClient.connectedNodes());
+ }
+
+ return transportClient;
+ }
+ }
+
+ @Override
+ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) {
+ if (!bulkItemResponse.isFailed()) {
+ return null;
+ } else {
+ return new RuntimeException(bulkItemResponse.getFailureMessage());
+ }
+ }
+
+ @Override
+ public void cleanup() {
+ if (node != null && !node.isClosed()) {
+ node.close();
+ node = null;
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
index ac14adeeae6c9..c3388606e0ee5 100644
--- a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
+++ b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSink.java
@@ -17,59 +17,38 @@
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.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.action.ActionRequest;
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.transport.TransportClient;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.collect.ImmutableList;
-import org.elasticsearch.common.settings.ImmutableSettings;
-import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
-import org.elasticsearch.common.unit.ByteSizeUnit;
-import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.node.Node;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
-
/**
- * Sink that emits its input elements to an Elasticsearch cluster.
+ * Elasticsearch 1.x sink that requests multiple {@link ActionRequest ActionRequests}
+ * against a cluster for each incoming element.
*
*
- * When using the first constructor {@link #ElasticsearchSink(java.util.Map, IndexRequestBuilder)}
- * the sink will create a local {@link Node} for communicating with the
- * Elasticsearch cluster. When using the second constructor
- * {@link #ElasticsearchSink(java.util.Map, java.util.List, IndexRequestBuilder)} a {@link TransportClient} will
- * be used instead.
+ * When using the first constructor {@link #ElasticsearchSink(java.util.Map, ElasticsearchSinkFunction)}
+ * the sink will create a local {@link Node} for communicating with the Elasticsearch cluster. When using the second
+ * constructor {@link #ElasticsearchSink(java.util.Map, java.util.List, ElasticsearchSinkFunction)} a
+ * {@link TransportClient} will be used instead.
*
*
* Attention: When using the {@code TransportClient} the sink will fail if no cluster
- * can be connected to. With the {@code Node Client} the sink will block and wait for a cluster
+ * can be connected to. When using the local {@code Node} for communicating, the sink will block and wait for a cluster
* to come online.
*
*
- * The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
- * the {@link Node} or {@link TransportClient}. The config keys can be found in the Elasticsearch
- * documentation. An important setting is {@code cluster.name}, this should be set to the name
- * of the cluster that the sink should emit to.
+ * The {@link Map} passed to the constructor is used to create the {@link Node} or {@link 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 IndexRequest IndexRequests}.
+ * 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:
*
*
*
- * You also have to provide an {@link IndexRequestBuilder}. This is used to create an
- * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
- * {@link org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder} for an example.
+ * 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 emitted by this sink
+ * @param Type of the elements handled by this sink
*/
-public class ElasticsearchSink 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 class ElasticsearchSink extends ElasticsearchSinkBase {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
-
- /**
- * The user specified config map that we forward to Elasticsearch when we create the Client.
- */
- private final Map userConfig;
-
- /**
- * The list of nodes that the TransportClient should connect to. This is null if we are using
- * an embedded Node to get a Client.
- */
- private final List transportNodes;
-
- /**
- * The builder that is used to construct an {@link IndexRequest} from the incoming element.
- */
- private final IndexRequestBuilder indexRequestBuilder;
-
- /**
- * The embedded Node that is used to communicate with the Elasticsearch cluster. This is null
- * if we are using a TransportClient.
- */
- private transient Node node;
-
/**
- * The Client that was either retrieved from a Node or is a TransportClient.
- */
- private transient Client client;
-
- /**
- * Bulk processor that was created using the client
- */
- private transient BulkProcessor bulkProcessor;
-
- /**
- * This is set from inside the BulkProcessor listener if there where failures in processing.
- */
- private final AtomicBoolean hasFailure = new AtomicBoolean(false);
-
- /**
- * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
- */
- private final AtomicReference failureThrowable = new AtomicReference<>();
-
- /**
- * Creates a new ElasticsearchSink that connects to the cluster using an embedded Node.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using an embedded {@link Node}.
*
- * @param userConfig The map of user settings that are passed when constructing the Node and BulkProcessor
+ * @param userConfig The map of user settings that are used when constructing the {@link Node} and {@link BulkProcessor}
* @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
+ *
+ * @deprecated Deprecated since version 1.2, to be removed at version 2.0.
+ * Please use {@link ElasticsearchSink#ElasticsearchSink(Map, ElasticsearchSinkFunction)} instead.
*/
+ @Deprecated
public ElasticsearchSink(Map userConfig, IndexRequestBuilder indexRequestBuilder) {
- this.userConfig = userConfig;
- this.indexRequestBuilder = indexRequestBuilder;
- transportNodes = null;
+ this(userConfig, new IndexRequestBuilderWrapperFunction<>(indexRequestBuilder));
}
/**
- * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link TransportClient}.
*
- * @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor
- * @param transportNodes The Elasticsearch Nodes to which to connect using a {@code TransportClient}
- * @param indexRequestBuilder This is used to generate the IndexRequest from the incoming element
+ * @param userConfig The map of user settings that are used when constructing the {@link TransportClient} and {@link BulkProcessor}
+ * @param transportAddresses The addresses of Elasticsearch nodes to which to connect using a {@link TransportClient}
+ * @param indexRequestBuilder This is used to generate a {@link IndexRequest} from the incoming element
*
+ * @deprecated Deprecated since 1.2, to be removed at 2.0.
+ * Please use {@link ElasticsearchSink#ElasticsearchSink(Map, List, ElasticsearchSinkFunction)} instead.
*/
- public ElasticsearchSink(Map userConfig, List transportNodes, IndexRequestBuilder indexRequestBuilder) {
- this.userConfig = userConfig;
- this.indexRequestBuilder = indexRequestBuilder;
- this.transportNodes = transportNodes;
+ @Deprecated
+ public ElasticsearchSink(Map userConfig, List transportAddresses, IndexRequestBuilder indexRequestBuilder) {
+ this(userConfig, transportAddresses, new IndexRequestBuilderWrapperFunction<>(indexRequestBuilder));
}
/**
- * Initializes the connection to Elasticsearch by either creating an embedded
- * {@link org.elasticsearch.node.Node} and retrieving the
- * {@link org.elasticsearch.client.Client} from it or by creating a
- * {@link org.elasticsearch.client.transport.TransportClient}.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using an embedded {@link Node}.
+ *
+ * @param userConfig The map of user settings that are used when constructing the embedded {@link Node} and {@link BulkProcessor}
+ * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element
*/
- @Override
- public void open(Configuration configuration) {
- if (transportNodes == null) {
- // Make sure that we disable http access to our embedded node
- Settings settings =
- ImmutableSettings.settingsBuilder()
- .put(userConfig)
- .put("http.enabled", false)
- .build();
-
- node =
- nodeBuilder()
- .settings(settings)
- .client(true)
- .data(false)
- .node();
-
- client = node.client();
-
- if (LOG.isInfoEnabled()) {
- LOG.info("Created Elasticsearch Client {} from embedded Node", client);
- }
-
- } else {
- Settings settings = ImmutableSettings.settingsBuilder()
- .put(userConfig)
- .build();
-
- TransportClient transportClient = new TransportClient(settings);
- for (TransportAddress transport: transportNodes) {
- transportClient.addTransportAddress(transport);
- }
-
- // verify that we actually are connected to a cluster
- ImmutableList nodes = transportClient.connectedNodes();
- if (nodes.isEmpty()) {
- throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
- } else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Connected to nodes: " + nodes.toString());
- }
- }
-
- client = transportClient;
-
- if (LOG.isInfoEnabled()) {
- LOG.info("Created Elasticsearch TransportClient {}", client);
- }
- }
-
- 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()) {
- if (itemResp.isFailed()) {
- LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
- failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
- }
- }
- hasFailure.set(true);
- }
- }
-
- @Override
- public void afterBulk(long executionId,
- BulkRequest request,
- Throwable failure) {
- LOG.error(failure.getMessage());
- failureThrowable.compareAndSet(null, failure);
- hasFailure.set(true);
- }
- });
-
- // This makes flush() blocking
- bulkProcessorBuilder.setConcurrentRequests(0);
-
- ParameterTool params = ParameterTool.fromMap(userConfig);
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
- bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
- bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
- CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
- bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
- }
-
- bulkProcessor = bulkProcessorBuilder.build();
- }
-
- @Override
- public void invoke(T element) {
- IndexRequest indexRequest = indexRequestBuilder.createIndexRequest(element, getRuntimeContext());
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Emitting IndexRequest: {}", indexRequest);
- }
-
- bulkProcessor.add(indexRequest);
+ public ElasticsearchSink(Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ super(new Elasticsearch1ApiCallBridge(), userConfig, elasticsearchSinkFunction);
}
- @Override
- public void close() {
- if (bulkProcessor != null) {
- bulkProcessor.close();
- bulkProcessor = null;
- }
-
- if (client != null) {
- client.close();
- }
-
- if (node != null) {
- node.close();
- }
-
- if (hasFailure.get()) {
- Throwable cause = failureThrowable.get();
- if (cause != null) {
- throw new RuntimeException("An error occured in ElasticsearchSink.", cause);
- } else {
- throw new RuntimeException("An error occured in ElasticsearchSink.");
-
- }
- }
+ /**
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link TransportClient}.
+ *
+ * @param userConfig The map of user settings that are used when constructing the {@link TransportClient} and {@link BulkProcessor}
+ * @param transportAddresses The addresses of Elasticsearch nodes to which to connect using a {@link TransportClient}
+ * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element
+ */
+ public ElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ super(new Elasticsearch1ApiCallBridge(transportAddresses), userConfig, elasticsearchSinkFunction);
}
-
}
diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
index 04ae40a2cadee..18aa11e980abc 100644
--- a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
+++ b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilder.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -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,
@@ -51,7 +51,11 @@
* }
*
* @param The type of the element handled by this {@code IndexRequestBuilder}
+ *
+ * @deprecated Deprecated since version 1.2, to be removed at version 2.0.
+ * Please create a {@link ElasticsearchSink} using a {@link ElasticsearchSinkFunction} instead.
*/
+@Deprecated
public interface IndexRequestBuilder extends Function, Serializable {
/**
diff --git a/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilderWrapperFunction.java b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilderWrapperFunction.java
new file mode 100644
index 0000000000000..6f1d13845411b
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/IndexRequestBuilderWrapperFunction.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;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+
+/**
+ * A dummy {@link ElasticsearchSinkFunction} that wraps a {@link IndexRequestBuilder}.
+ * This serves as a bridge for the usage deprecation of the {@code IndexRequestBuilder} interface.
+ */
+class IndexRequestBuilderWrapperFunction implements ElasticsearchSinkFunction {
+
+ private static final long serialVersionUID = 289876038414250101L;
+
+ private final IndexRequestBuilder indexRequestBuilder;
+
+ IndexRequestBuilderWrapperFunction(IndexRequestBuilder indexRequestBuilder) {
+ this.indexRequestBuilder = indexRequestBuilder;
+ }
+
+ @Override
+ public void process(T element, RuntimeContext ctx, RequestIndexer indexer) {
+ indexer.add(indexRequestBuilder.createIndexRequest(element, ctx));
+ }
+}
diff --git a/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
index 33a2e476e9ae7..3a7b11360c9cf 100644
--- a/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
+++ b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkITCase.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -18,177 +18,149 @@
package org.apache.flink.streaming.connectors.elasticsearch;
import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
import org.apache.flink.api.common.functions.RuntimeContext;
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.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.elasticsearch.action.get.GetRequest;
-import org.elasticsearch.action.get.GetResponse;
+import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit;
+import org.apache.flink.streaming.connectors.elasticsearch.util.ElasticsearchUtils;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
-import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
-import org.elasticsearch.node.Node;
-import org.junit.Assert;
-import org.junit.ClassRule;
import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import java.io.File;
+import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase {
-public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
+ @Test
+ public void testTransportClient() throws Exception {
+ runTransportClientTest();
+ }
- private static final int NUM_ELEMENTS = 20;
+ @Test
+ public void testNullTransportClient() throws Exception {
+ runNullTransportClientTest();
+ }
- @ClassRule
- public static TemporaryFolder tempFolder = new TemporaryFolder();
+ @Test
+ public void testEmptyTransportClient() throws Exception {
+ runEmptyTransportClientTest();
+ }
@Test
- public void testNodeClient() throws Exception{
+ public void testTransportClientFails() throws Exception{
+ runTransportClientFailsTest();
+ }
- File dataDir = tempFolder.newFolder();
+ // -- Tests specific to Elasticsearch 1.x --
- Node node = nodeBuilder()
- .settings(ImmutableSettings.settingsBuilder()
- .put("http.enabled", false)
- .put("path.data", dataDir.getAbsolutePath()))
- // set a custom cluster name to verify that user config works correctly
- .clusterName("my-node-client-cluster")
- .local(true)
- .node();
+ /**
+ * Tests that the Elasticsearch sink works properly using an embedded node to connect to Elasticsearch.
+ */
+ @Test
+ public void testEmbeddedNode() throws Exception {
+ final String index = "embedded-node-test-index";
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataStreamSource> source = env.addSource(new TestSourceFunction());
+ DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction());
- Map config = Maps.newHashMap();
+ Map userConfig = new HashMap<>();
// This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-node-client-cluster");
-
- // connect to our local node
- config.put("node.local", "true");
+ userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+ userConfig.put("cluster.name", CLUSTER_NAME);
+ userConfig.put("node.local", "true");
- source.addSink(new ElasticsearchSink<>(config, new TestIndexRequestBuilder()));
-
- env.execute("Elasticsearch Node Client Test");
+ source.addSink(new ElasticsearchSink<>(
+ userConfig,
+ new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))
+ );
+ env.execute("Elasticsearch Embedded Node Test");
// verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.get(new GetRequest("my-index",
- "my-type",
- Integer.toString(i))).actionGet();
- Assert.assertEquals("message #" + i, response.getSource().get("data"));
- }
+ Client client = embeddedNodeEnv.getClient();
+ SourceSinkDataTestKit.verifyProducedSinkData(client, index);
- node.close();
+ client.close();
}
+ /**
+ * Tests that behaviour of the deprecated {@link IndexRequestBuilder} constructor works properly.
+ */
@Test
- public void testTransportClient() throws Exception {
-
- File dataDir = tempFolder.newFolder();
+ public void testDeprecatedIndexRequestBuilderVariant() throws Exception {
+ final String index = "index-req-builder-test-index";
- Node node = nodeBuilder()
- .settings(ImmutableSettings.settingsBuilder()
- .put("http.enabled", false)
- .put("path.data", dataDir.getAbsolutePath()))
- // set a custom cluster name to verify that user config works correctly
- .clusterName("my-node-client-cluster")
- .local(true)
- .node();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction());
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map config = Maps.newHashMap();
+ Map userConfig = new HashMap<>();
// This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-node-client-cluster");
-
- // connect to our local node
- config.put("node.local", "true");
+ userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+ userConfig.put("cluster.name", CLUSTER_NAME);
+ userConfig.put("node.local", "true");
List transports = Lists.newArrayList();
transports.add(new LocalTransportAddress("1"));
- source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
-
- env.execute("Elasticsearch TransportClient Test");
+ source.addSink(new ElasticsearchSink<>(
+ userConfig,
+ transports,
+ new TestIndexRequestBuilder(index))
+ );
+ env.execute("Elasticsearch Deprecated IndexRequestBuilder Bridge Test");
// verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.get(new GetRequest("my-index",
- "my-type",
- Integer.toString(i))).actionGet();
- Assert.assertEquals("message #" + i, response.getSource().get("data"));
- }
+ Client client = embeddedNodeEnv.getClient();
+ SourceSinkDataTestKit.verifyProducedSinkData(client, index);
- node.close();
+ client.close();
}
- @Test(expected = JobExecutionException.class)
- public void testTransportClientFails() throws Exception{
- // this checks whether the TransportClient fails early when there is no cluster to
- // connect to. We don't hava such as test for the Node Client version since that
- // one will block and wait for a cluster to come online
-
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig,
+ List transportAddresses,
+ ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ return new ElasticsearchSink<>(userConfig, ElasticsearchUtils.convertInetSocketAddresses(transportAddresses), elasticsearchSinkFunction);
+ }
- Map config = Maps.newHashMap();
- // This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-node-client-cluster");
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(
+ Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception {
- // connect to our local node
- config.put("node.local", "true");
+ // Elasticsearch 1.x requires this setting when using
+ // LocalTransportAddress to connect to a local embedded node
+ userConfig.put("node.local", "true");
List transports = Lists.newArrayList();
transports.add(new LocalTransportAddress("1"));
- source.addSink(new ElasticsearchSink<>(config, transports, new TestIndexRequestBuilder()));
-
- env.execute("Elasticsearch Node Client Test");
+ return new ElasticsearchSink<>(
+ userConfig,
+ transports,
+ elasticsearchSinkFunction);
}
- private static class TestSourceFunction implements SourceFunction> {
+ /**
+ * A {@link IndexRequestBuilder} with equivalent functionality to {@link SourceSinkDataTestKit.TestElasticsearchSinkFunction}.
+ */
+ private static class TestIndexRequestBuilder implements IndexRequestBuilder> {
private static final long serialVersionUID = 1L;
- private volatile boolean running = true;
+ private final String index;
- @Override
- public void run(SourceContext> ctx) throws Exception {
- for (int i = 0; i < NUM_ELEMENTS && running; i++) {
- ctx.collect(Tuple2.of(i, "message #" + i));
- }
- }
-
- @Override
- public void cancel() {
- running = false;
+ public TestIndexRequestBuilder(String index) {
+ this.index = index;
}
- }
-
- private static class TestIndexRequestBuilder implements IndexRequestBuilder> {
- private static final long serialVersionUID = 1L;
@Override
public IndexRequest createIndexRequest(Tuple2 element, RuntimeContext ctx) {
@@ -196,10 +168,10 @@ public IndexRequest createIndexRequest(Tuple2 element, RuntimeC
json.put("data", element.f1);
return Requests.indexRequest()
- .index("my-index")
- .type("my-type")
- .id(element.f0.toString())
- .source(json);
+ .index(index)
+ .type("flink-es-test-type")
+ .id(element.f0.toString())
+ .source(json);
}
}
}
diff --git a/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
new file mode 100644
index 0000000000000..a0c809b50f101
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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 org.elasticsearch.common.settings.ImmutableSettings;
+import org.elasticsearch.node.Node;
+
+import java.io.File;
+
+import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
+
+/**
+ * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 1.x.
+ * 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) {
+ node = nodeBuilder()
+ .settings(ImmutableSettings.settingsBuilder()
+ .put("http.enabled", false)
+ .put("path.data", tmpDataFolder.getAbsolutePath()))
+ .clusterName(clusterName)
+ .local(true)
+ .node();
+
+ 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;
+ }
+ }
+}
diff --git a/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchSinkExample.java
similarity index 52%
rename from flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
rename to flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchSinkExample.java
index 136ae775ea75f..d697c3c7b4005 100644
--- a/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchExample.java
+++ b/flink-connectors/flink-connector-elasticsearch/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/examples/ElasticsearchSinkExample.java
@@ -17,64 +17,68 @@
package org.apache.flink.streaming.connectors.elasticsearch.examples;
+import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink;
-import org.apache.flink.streaming.connectors.elasticsearch.IndexRequestBuilder;
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction;
+import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Requests;
+import org.elasticsearch.common.transport.InetSocketTransportAddress;
+import org.elasticsearch.common.transport.TransportAddress;
+import java.net.InetAddress;
+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 cluster name in the config map.
*/
-public class ElasticsearchExample {
+public class ElasticsearchSinkExample {
public static void main(String[] args) throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataStreamSource source = env.addSource(new SourceFunction() {
- private static final long serialVersionUID = 1L;
-
- private volatile boolean running = true;
-
- @Override
- public void run(SourceContext ctx) throws Exception {
- for (int i = 0; i < 20 && running; i++) {
- ctx.collect("message #" + i);
- }
- }
-
+ DataStream source = env.generateSequence(0, 20).map(new MapFunction() {
@Override
- public void cancel() {
- running = false;
+ public String map(Long value) throws Exception {
+ return "message #" + value;
}
});
- Map config = new HashMap<>();
+ Map userConfig = new HashMap<>();
+ userConfig.put("cluster.name", "elasticsearch");
// This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+ userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- source.addSink(new ElasticsearchSink<>(config, new IndexRequestBuilder() {
- @Override
- public IndexRequest createIndexRequest(String element, RuntimeContext ctx) {
- Map json = new HashMap<>();
- json.put("data", element);
+ List transports = new ArrayList<>();
+ transports.add(new InetSocketTransportAddress(InetAddress.getByName("127.0.0.1"), 9300));
- return Requests.indexRequest()
- .index("my-index")
- .type("my-type")
- .source(json);
+ source.addSink(new ElasticsearchSink<>(userConfig, transports, new ElasticsearchSinkFunction() {
+ @Override
+ public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
+ indexer.add(createIndexRequest(element));
}
}));
- env.execute("Elasticsearch Example");
+ 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-elasticsearch/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
index dc207262c0760..20551848eea24 100644
--- a/flink-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
+++ b/flink-connectors/flink-connector-elasticsearch/src/test/resources/log4j-test.properties
@@ -16,12 +16,12 @@
# limitations under the License.
################################################################################
-log4j.rootLogger=OFF, testlogger
+log4j.rootLogger=INFO, testlogger
log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
+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
\ No newline at end of file
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
diff --git a/flink-connectors/flink-connector-elasticsearch2/pom.xml b/flink-connectors/flink-connector-elasticsearch2/pom.xml
index 7aba36e7872f9..30396ddf38013 100644
--- a/flink-connectors/flink-connector-elasticsearch2/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch2/pom.xml
@@ -52,17 +52,19 @@ under the License.
- org.elasticsearch
- elasticsearch
- ${elasticsearch.version}
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
+
- com.fasterxml.jackson.core
- jackson-core
+ org.elasticsearch
+ elasticsearch
+ ${elasticsearch.version}
-
+
org.apache.flink
@@ -78,6 +80,15 @@ under the License.
test
test-jar
+
+
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
+ test-jar
+ test
+
+
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2ApiCallBridge.java
new file mode 100644
index 0000000000000..9407d9f628127
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/Elasticsearch2ApiCallBridge.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.elasticsearch2;
+
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge;
+import org.apache.flink.streaming.connectors.elasticsearch.util.ElasticsearchUtils;
+import org.apache.flink.util.Preconditions;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 2.x.
+ */
+public class Elasticsearch2ApiCallBridge implements ElasticsearchApiCallBridge {
+
+ private static final long serialVersionUID = 2638252694744361079L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch2ApiCallBridge.class);
+
+ /**
+ * User-provided transport addresses.
+ *
+ * We are using {@link InetSocketAddress} because {@link TransportAddress} is not serializable in Elasticsearch 2.x.
+ */
+ private final List transportAddresses;
+
+ Elasticsearch2ApiCallBridge(List transportAddresses) {
+ Preconditions.checkArgument(transportAddresses != null && !transportAddresses.isEmpty());
+ this.transportAddresses = transportAddresses;
+ }
+
+ @Override
+ public Client createClient(Map clientConfig) {
+ Settings settings = Settings.settingsBuilder().put(clientConfig).build();
+
+ TransportClient transportClient = TransportClient.builder().settings(settings).build();
+ for (TransportAddress address : ElasticsearchUtils.convertInetSocketAddresses(transportAddresses)) {
+ transportClient.addTransportAddress(address);
+ }
+
+ // verify that we actually are connected to a cluster
+ if (transportClient.connectedNodes().isEmpty()) {
+ throw new RuntimeException("Elasticsearch client is not connected to any Elasticsearch nodes!");
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Created Elasticsearch TransportClient with connected nodes {}", transportClient.connectedNodes());
+ }
+
+ return transportClient;
+ }
+
+ @Override
+ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) {
+ if (!bulkItemResponse.isFailed()) {
+ return null;
+ } else {
+ return bulkItemResponse.getFailure().getCause();
+ }
+ }
+
+ @Override
+ public void cleanup() {
+ // nothing to cleanup
+ }
+
+}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java
index e839589690c96..a0abc51bd6c3d 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java
@@ -16,55 +16,30 @@
*/
package org.apache.flink.streaming.connectors.elasticsearch2;
-import com.google.common.collect.ImmutableList;
-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.Preconditions;
-import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase;
+import org.elasticsearch.action.ActionRequest;
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.transport.TransportClient;
-import org.elasticsearch.cluster.node.DiscoveryNode;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.transport.TransportAddress;
-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.net.InetSocketAddress;
-import java.util.ArrayList;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
/**
- * Sink that emits its input elements in bulk to an Elasticsearch cluster.
+ * Elasticsearch 2.x sink that requests multiple {@link ActionRequest ActionRequests}
+ * against a cluster for each incoming element.
*
*
- * When using the second constructor
- * {@link #ElasticsearchSink(java.util.Map, java.util.List, ElasticsearchSinkFunction)} a {@link TransportClient} will
- * be used.
+ * The sink internally uses a {@link TransportClient} 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.
*
*
- * Attention: When using the {@code TransportClient} the sink will fail if no cluster
- * can be connected to.
+ * 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.
*
*
- * The {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
- * {@link TransportClient}. The config keys can be found in the Elasticsearch
- * documentation. An important setting is {@code cluster.name}, this 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 IndexRequest IndexRequests}.
+ * 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:
*
*
*
- * You also have to provide an {@link RequestIndexer}. This is used to create an
- * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
- * {@link RequestIndexer} for an example.
+ * You also have to provide an {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction}.
+ * This is used to create multiple {@link ActionRequest ActionRequests} for each incoming element. See the class level
+ * documentation of {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction} for an example.
*
- * @param Type of the elements emitted by this sink
+ * @param Type of the elements handled by this sink
*/
-public class ElasticsearchSink 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 class ElasticsearchSink extends ElasticsearchSinkBase {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
-
- /**
- * The user specified config map that we forward to Elasticsearch when we create the Client.
- */
- private final Map userConfig;
-
- /**
- * The list of nodes that the TransportClient should connect to. This is null if we are using
- * an embedded Node to get a Client.
- */
- private final List transportAddresses;
-
- /**
- * The builder that is used to construct an {@link IndexRequest} from the incoming element.
- */
- private final ElasticsearchSinkFunction elasticsearchSinkFunction;
-
- /**
- * The Client that was either retrieved from a Node or is a TransportClient.
- */
- private transient Client client;
-
- /**
- * Bulk processor that was created using the client
- */
- private transient BulkProcessor bulkProcessor;
-
- /**
- * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
- */
- private transient RequestIndexer requestIndexer;
-
- /**
- * This is set from inside the BulkProcessor listener if there where failures in processing.
- */
- private final AtomicBoolean hasFailure = new AtomicBoolean(false);
-
- /**
- * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
- */
- private final AtomicReference failureThrowable = new AtomicReference<>();
-
/**
- * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link TransportClient}.
*
- * @param userConfig The map of user settings that are passed when constructing the TransportClient and BulkProcessor
- * @param transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
- * @param elasticsearchSinkFunction This is used to generate the ActionRequest from the incoming element
+ * @param userConfig The map of user settings that are used when constructing the {@link TransportClient} and {@link BulkProcessor}
+ * @param transportAddresses The addresses of Elasticsearch nodes to which to connect using a {@link TransportClient}
+ * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element
*
+ * @deprecated Deprecated since 1.2, to be removed at 2.0.
+ * Please use {@link ElasticsearchSink#ElasticsearchSink(Map, List, org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction)} instead.
*/
+ @Deprecated
public ElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) {
- this.userConfig = userConfig;
- this.elasticsearchSinkFunction = elasticsearchSinkFunction;
- Preconditions.checkArgument(transportAddresses != null && transportAddresses.size() > 0);
- this.transportAddresses = transportAddresses;
+ this(userConfig, transportAddresses, new OldNewElasticsearchSinkFunctionBridge<>(elasticsearchSinkFunction));
}
/**
- * Initializes the connection to Elasticsearch by creating a
- * {@link org.elasticsearch.client.transport.TransportClient}.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link TransportClient}.
+ *
+ * @param userConfig The map of user settings that are passed when constructing the {@link TransportClient} and {@link BulkProcessor}
+ * @param transportAddresses The addresses of Elasticsearch nodes to which to connect using a {@link TransportClient}
+ * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element
*/
- @Override
- public void open(Configuration configuration) {
- List transportNodes;
- transportNodes = new ArrayList<>(transportAddresses.size());
- for (InetSocketAddress address : transportAddresses) {
- transportNodes.add(new InetSocketTransportAddress(address));
- }
-
- Settings settings = Settings.settingsBuilder().put(userConfig).build();
-
- TransportClient transportClient = TransportClient.builder().settings(settings).build();
- for (TransportAddress transport: transportNodes) {
- transportClient.addTransportAddress(transport);
- }
-
- // verify that we actually are connected to a cluster
- ImmutableList nodes = ImmutableList.copyOf(transportClient.connectedNodes());
- if (nodes.isEmpty()) {
- throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
- }
-
- client = transportClient;
-
- if (LOG.isInfoEnabled()) {
- LOG.info("Created Elasticsearch TransportClient {}", client);
- }
-
- 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()) {
- if (itemResp.isFailed()) {
- LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
- failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
- }
- }
- hasFailure.set(true);
- }
- }
-
- @Override
- public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
- LOG.error(failure.getMessage());
- failureThrowable.compareAndSet(null, failure);
- hasFailure.set(true);
- }
- });
-
- // This makes flush() blocking
- bulkProcessorBuilder.setConcurrentRequests(0);
-
- ParameterTool params = ParameterTool.fromMap(userConfig);
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
- bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
- bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
- CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
- bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
- }
-
- bulkProcessor = bulkProcessorBuilder.build();
- requestIndexer = new BulkProcessorIndexer(bulkProcessor);
- }
-
- @Override
- public void invoke(T element) {
- elasticsearchSinkFunction.process(element, getRuntimeContext(), requestIndexer);
+ public ElasticsearchSink(Map userConfig,
+ List transportAddresses,
+ org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ super(new Elasticsearch2ApiCallBridge(transportAddresses), userConfig, elasticsearchSinkFunction);
}
-
- @Override
- public void close() {
- if (bulkProcessor != null) {
- bulkProcessor.close();
- bulkProcessor = null;
- }
-
- if (client != null) {
- client.close();
- }
-
- if (hasFailure.get()) {
- Throwable cause = failureThrowable.get();
- if (cause != null) {
- throw new RuntimeException("An error occured in ElasticsearchSink.", cause);
- } else {
- throw new RuntimeException("An error occured in ElasticsearchSink.");
- }
- }
-
- }
-
}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
index 55ba720d82d6c..c474390239270 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkFunction.java
@@ -1,13 +1,12 @@
-/**
- * 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,
@@ -23,7 +22,7 @@
import java.io.Serializable;
/**
- * Method that creates an {@link org.elasticsearch.action.ActionRequest} from an element in a Stream.
+ * Method that creates multiple {@link org.elasticsearch.action.ActionRequest}s from an element in a Stream.
*
*
* This is used by {@link ElasticsearchSink} to prepare elements for sending them to Elasticsearch.
@@ -54,7 +53,12 @@
* }
*
* @param The type of the element handled by this {@code ElasticsearchSinkFunction}
+ *
+ * @deprecated Deprecated since 1.2, to be removed at 2.0.
+ * This class has been deprecated due to package relocation.
+ * Please use {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction} instead.
*/
+@Deprecated
public interface ElasticsearchSinkFunction extends Serializable, Function {
void process(T element, RuntimeContext ctx, RequestIndexer indexer);
}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewElasticsearchSinkFunctionBridge.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewElasticsearchSinkFunctionBridge.java
new file mode 100644
index 0000000000000..c95fff5d6fd69
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewElasticsearchSinkFunctionBridge.java
@@ -0,0 +1,45 @@
+/*
+ * 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.elasticsearch2;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer;
+
+/**
+ * A dummy {@link org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction} to bridge
+ * the migration from the deprecated {@link ElasticsearchSinkFunction}.
+ */
+class OldNewElasticsearchSinkFunctionBridge implements org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction {
+
+ private static final long serialVersionUID = 2415651895272659448L;
+
+ private final ElasticsearchSinkFunction deprecated;
+ private OldNewRequestIndexerBridge reusedRequestIndexerBridge;
+
+ OldNewElasticsearchSinkFunctionBridge(ElasticsearchSinkFunction deprecated) {
+ this.deprecated = deprecated;
+ }
+
+ @Override
+ public void process(T element, RuntimeContext ctx, RequestIndexer indexer) {
+ if (reusedRequestIndexerBridge == null) {
+ reusedRequestIndexerBridge = new OldNewRequestIndexerBridge(indexer);
+ }
+ deprecated.process(element, ctx, reusedRequestIndexerBridge);
+ }
+}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewRequestIndexerBridge.java
similarity index 62%
rename from flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
rename to flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewRequestIndexerBridge.java
index 650931f26f97a..f42fb44759860 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/BulkProcessorIndexer.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/OldNewRequestIndexerBridge.java
@@ -14,22 +14,28 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.flink.streaming.connectors.elasticsearch2;
import org.elasticsearch.action.ActionRequest;
-import org.elasticsearch.action.bulk.BulkProcessor;
-public class BulkProcessorIndexer implements RequestIndexer {
- private final BulkProcessor bulkProcessor;
+/**
+ * A dummy {@link org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer} to bridge
+ * the migration from the deprecated {@link RequestIndexer}.
+ */
+class OldNewRequestIndexerBridge implements RequestIndexer {
+
+ private static final long serialVersionUID = 4213982619497149416L;
- public BulkProcessorIndexer(BulkProcessor bulkProcessor) {
- this.bulkProcessor = bulkProcessor;
+ private final org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer requestIndexer;
+
+ OldNewRequestIndexerBridge(org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer requestIndexer) {
+ this.requestIndexer = requestIndexer;
}
@Override
public void add(ActionRequest... actionRequests) {
- for (ActionRequest actionRequest : actionRequests) {
- this.bulkProcessor.add(actionRequest);
- }
+ requestIndexer.add(actionRequests);
}
+
}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
index 144a87b0b9a89..b2b3de4acc584 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/RequestIndexer.java
@@ -20,6 +20,12 @@
import java.io.Serializable;
+/**
+ * @deprecated Deprecated since 1.2, to be removed at 2.0.
+ * This class has been deprecated due to package relocation.
+ * Please use {@link org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer} instead.
+ */
+@Deprecated
public interface RequestIndexer extends Serializable {
void add(ActionRequest... actionRequests);
-}
\ No newline at end of file
+}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
new file mode 100644
index 0000000000000..ddf3bd62a56de
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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.elasticsearch2.ElasticsearchSinkITCase;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.NodeBuilder;
+
+import java.io.File;
+
+/**
+ * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 2.x.
+ * 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) {
+ node = NodeBuilder.nodeBuilder().settings(
+ Settings.settingsBuilder()
+ .put("path.home", tmpDataFolder.getParent())
+ .put("http.enabled", false)
+ .put("path.data", tmpDataFolder.getAbsolutePath()))
+ .clusterName(clusterName)
+ .node();
+
+ 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;
+ }
+ }
+
+}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
index bc9bedcdc5d32..93ac6c8ed2f6d 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSinkITCase.java
@@ -1,13 +1,12 @@
-/**
- * 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,
@@ -17,217 +16,53 @@
*/
package org.apache.flink.streaming.connectors.elasticsearch2;
-import org.apache.flink.api.common.functions.RuntimeContext;
-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.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-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.Requests;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.node.NodeBuilder;
-import org.junit.Assert;
-import org.junit.ClassRule;
+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.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import java.io.File;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.ArrayList;
-import java.util.HashMap;
import java.util.List;
import java.util.Map;
-public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
-
- private static final int NUM_ELEMENTS = 20;
-
- @ClassRule
- public static TemporaryFolder tempFolder = new TemporaryFolder();
+public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase {
@Test
public void testTransportClient() throws Exception {
-
- File dataDir = tempFolder.newFolder();
-
- Node node = NodeBuilder.nodeBuilder()
- .settings(Settings.settingsBuilder()
- .put("path.home", dataDir.getParent())
- .put("http.enabled", false)
- .put("path.data", dataDir.getAbsolutePath()))
- // set a custom cluster name to verify that user config works correctly
- .clusterName("my-transport-client-cluster")
- .node();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map config = new HashMap<>();
- // This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-transport-client-cluster");
-
- // Can't use {@link TransportAddress} as its not Serializable in Elasticsearch 2.x
- List transports = new ArrayList<>();
- transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
- source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch TransportClient Test");
-
- // verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.get(new GetRequest("my-index",
- "my-type", Integer.toString(i))).actionGet();
- Assert.assertEquals("message #" + i, response.getSource().get("data"));
- }
-
- node.close();
+ runTransportClientTest();
}
- @Test(expected = IllegalArgumentException.class)
- public void testNullTransportClient() throws Exception {
-
- File dataDir = tempFolder.newFolder();
-
- Node node = NodeBuilder.nodeBuilder()
- .settings(Settings.settingsBuilder()
- .put("path.home", dataDir.getParent())
- .put("http.enabled", false)
- .put("path.data", dataDir.getAbsolutePath()))
- // set a custom cluster name to verify that user config works correctly
- .clusterName("my-transport-client-cluster")
- .node();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map config = new HashMap<>();
- // This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-transport-client-cluster");
-
- source.addSink(new ElasticsearchSink<>(config, null, new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch TransportClient Test");
-
- // verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.get(new GetRequest("my-index",
- "my-type", Integer.toString(i))).actionGet();
- Assert.assertEquals("message #" + i, response.getSource().get("data"));
+ @Test
+ public void testNullTransportClient() throws Exception {
+ runNullTransportClientTest();
}
- node.close();
- }
-
- @Test(expected = IllegalArgumentException.class)
- public void testEmptyTransportClient() throws Exception {
-
- File dataDir = tempFolder.newFolder();
-
- Node node = NodeBuilder.nodeBuilder()
- .settings(Settings.settingsBuilder()
- .put("path.home", dataDir.getParent())
- .put("http.enabled", false)
- .put("path.data", dataDir.getAbsolutePath()))
- // set a custom cluster name to verify that user config works correctly
- .clusterName("my-transport-client-cluster")
- .node();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map config = new HashMap<>();
- // This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-transport-client-cluster");
-
- source.addSink(new ElasticsearchSink<>(config, new ArrayList(), new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch TransportClient Test");
-
- // verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.get(new GetRequest("my-index",
- "my-type", Integer.toString(i))).actionGet();
- Assert.assertEquals("message #" + i, response.getSource().get("data"));
+ @Test
+ public void testEmptyTransportClient() throws Exception {
+ runEmptyTransportClientTest();
}
- node.close();
- }
-
- @Test(expected = JobExecutionException.class)
+ @Test
public void testTransportClientFails() throws Exception{
- // this checks whether the TransportClient fails early when there is no cluster to
- // connect to. There isn't a similar test for the Node Client version since that
- // one will block and wait for a cluster to come online
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map config = new HashMap<>();
- // This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
- config.put("cluster.name", "my-node-client-cluster");
-
- List transports = new ArrayList<>();
- transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
- source.addSink(new ElasticsearchSink<>(config, transports, new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch Node Client Test");
+ runTransportClientFailsTest();
}
- private static class TestSourceFunction implements SourceFunction> {
- private static final long serialVersionUID = 1L;
-
- private volatile boolean running = true;
-
- @Override
- public void run(SourceContext> ctx) throws Exception {
- for (int i = 0; i < NUM_ELEMENTS && running; i++) {
- ctx.collect(Tuple2.of(i, "message #" + i));
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig,
+ List transportAddresses,
+ ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ return new ElasticsearchSink<>(userConfig, transportAddresses, elasticsearchSinkFunction);
}
- private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> {
- private static final long serialVersionUID = 1L;
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(
+ Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception {
- 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);
- }
+ List transports = new ArrayList<>();
+ transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
- @Override
- public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
- indexer.add(createIndexRequest(element));
- }
+ return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction);
}
}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchSinkExample.java
similarity index 63%
rename from flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
rename to flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchSinkExample.java
index 05760e8411030..8c50847cf871a 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchExample.java
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/java/org/apache/flink/streaming/connectors/elasticsearch2/examples/ElasticsearchSinkExample.java
@@ -18,11 +18,9 @@
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink;
-import org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSinkFunction;
-import org.apache.flink.streaming.connectors.elasticsearch2.RequestIndexer;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Requests;
@@ -37,44 +35,35 @@
* 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 ElasticsearchExample {
+public class ElasticsearchSinkExample {
public static void main(String[] args) throws Exception {
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- SingleOutputStreamOperator source =
- env.generateSequence(0, 20).map(new MapFunction() {
- /**
- * The mapping method. Takes an element from the input data set and transforms
- * it into exactly one element.
- *
- * @param value The input value.
- * @return The transformed value
- * @throws Exception This method may throw exceptions. Throwing an exception will cause the operation
- * to fail and may trigger recovery.
- */
- @Override
- public String map(Long value) throws Exception {
- return "message #" + value;
- }
- });
+ DataStream source = env.generateSequence(0, 20).map(new MapFunction() {
+ @Override
+ public String map(Long value) throws Exception {
+ return "message #" + value;
+ }
+ });
- Map config = new HashMap<>();
+ Map userConfig = new HashMap<>();
+ userConfig.put("cluster.name", "elasticsearch");
// This instructs the sink to emit after every element, otherwise they would be buffered
- config.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+ userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
List transports = new ArrayList<>();
transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
- source.addSink(new ElasticsearchSink<>(config, transports, new ElasticsearchSinkFunction(){
+ source.addSink(new ElasticsearchSink<>(userConfig, transports, new org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction(){
@Override
- public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
+ public void process(String element, RuntimeContext ctx, org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer indexer) {
indexer.add(createIndexRequest(element));
}
}));
- env.execute("Elasticsearch Example");
+ env.execute("Elasticsearch Sink Example");
}
private static IndexRequest createIndexRequest(String element) {
@@ -82,9 +71,9 @@ private static IndexRequest createIndexRequest(String element) {
json.put("data", element);
return Requests.indexRequest()
- .index("my-index")
- .type("my-type")
- .id(element)
- .source(json);
+ .index("my-index")
+ .type("my-type")
+ .id(element)
+ .source(json);
}
}
diff --git a/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j-test.properties
index dc207262c0760..20551848eea24 100644
--- a/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j-test.properties
+++ b/flink-connectors/flink-connector-elasticsearch2/src/test/resources/log4j-test.properties
@@ -16,12 +16,12 @@
# limitations under the License.
################################################################################
-log4j.rootLogger=OFF, testlogger
+log4j.rootLogger=INFO, testlogger
log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
-log4j.appender.testlogger.target = System.err
+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
\ No newline at end of file
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger
diff --git a/flink-connectors/flink-connector-elasticsearch5/pom.xml b/flink-connectors/flink-connector-elasticsearch5/pom.xml
index 8fc5c8b892ad9..a0bd32880f7bd 100644
--- a/flink-connectors/flink-connector-elasticsearch5/pom.xml
+++ b/flink-connectors/flink-connector-elasticsearch5/pom.xml
@@ -26,7 +26,7 @@ under the License.
org.apache.flink
flink-connectors
- 1.2-SNAPSHOT
+ 1.3-SNAPSHOT
..
@@ -37,7 +37,7 @@ under the License.
- 5.0.0
+ 5.1.2
@@ -51,28 +51,39 @@ under the License.
provided
+
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
+
+
+
+ org.elasticsearch
+ elasticsearch
+
+
+
+
+
org.elasticsearch.client
transport
${elasticsearch.version}
+
+
org.apache.logging.log4j
- log4j-api
- 2.7
-
-
- org.apache.logging.log4j
- log4j-core
+ log4j-to-slf4j
2.7
-
- com.fasterxml.jackson.core
- jackson-core
-
-
@@ -81,6 +92,7 @@ under the License.
${project.version}
test
+
org.apache.flink
flink-streaming-java_2.10
@@ -88,6 +100,63 @@ under the License.
test
test-jar
+
+
+ org.apache.flink
+ flink-connector-elasticsearch-base_2.10
+ ${project.version}
+
+
+ org.elasticsearch
+ elasticsearch
+
+
+ test-jar
+ 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-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/BulkProcessorIndexer.java
deleted file mode 100644
index f7ca499b6a5d7..0000000000000
--- a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/BulkProcessorIndexer.java
+++ /dev/null
@@ -1,35 +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.elasticsearch5;
-
-import org.elasticsearch.action.ActionRequest;
-import org.elasticsearch.action.bulk.BulkProcessor;
-
-public class BulkProcessorIndexer implements RequestIndexer {
- private final BulkProcessor bulkProcessor;
-
- public 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-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java
new file mode 100644
index 0000000000000..1389e7d11fd4b
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/Elasticsearch5ApiCallBridge.java
@@ -0,0 +1,97 @@
+/*
+ * 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.elasticsearch5;
+
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge;
+import org.apache.flink.streaming.connectors.elasticsearch.util.ElasticsearchUtils;
+import org.apache.flink.util.Preconditions;
+import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.client.transport.TransportClient;
+import org.elasticsearch.common.network.NetworkModule;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.transport.TransportAddress;
+import org.elasticsearch.transport.Netty3Plugin;
+import org.elasticsearch.transport.client.PreBuiltTransportClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 5.x.
+ */
+public class Elasticsearch5ApiCallBridge implements ElasticsearchApiCallBridge {
+
+ private static final long serialVersionUID = -5222683870097809633L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch5ApiCallBridge.class);
+
+ /**
+ * User-provided transport addresses.
+ *
+ * We are using {@link InetSocketAddress} because {@link TransportAddress} is not serializable in Elasticsearch 5.x.
+ */
+ private final List transportAddresses;
+
+ Elasticsearch5ApiCallBridge(List transportAddresses) {
+ Preconditions.checkArgument(transportAddresses != null && !transportAddresses.isEmpty());
+ this.transportAddresses = transportAddresses;
+ }
+
+ @Override
+ public Client createClient(Map clientConfig) {
+ Settings settings = Settings.builder().put(clientConfig)
+ .put(NetworkModule.HTTP_TYPE_KEY, Netty3Plugin.NETTY_HTTP_TRANSPORT_NAME)
+ .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty3Plugin.NETTY_TRANSPORT_NAME)
+ .build();
+
+ TransportClient transportClient = new PreBuiltTransportClient(settings);
+ for (TransportAddress transport : ElasticsearchUtils.convertInetSocketAddresses(transportAddresses)) {
+ transportClient.addTransportAddress(transport);
+ }
+
+ // verify that we actually are connected to a cluster
+ if (transportClient.connectedNodes().isEmpty()) {
+ throw new RuntimeException("Elasticsearch client is not connected to any Elasticsearch nodes!");
+ }
+
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Created Elasticsearch TransportClient with connected nodes {}", transportClient.connectedNodes());
+ }
+
+ return transportClient;
+ }
+
+ @Override
+ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) {
+ if (!bulkItemResponse.isFailed()) {
+ return null;
+ } else {
+ return bulkItemResponse.getFailure().getCause();
+ }
+ }
+
+ @Override
+ public void cleanup() {
+ // nothing to cleanup
+ }
+
+}
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSink.java
index 29c69c40f3666..9107d4e5ba911 100644
--- a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSink.java
+++ b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSink.java
@@ -16,244 +16,61 @@
*/
package org.apache.flink.streaming.connectors.elasticsearch5;
-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.Preconditions;
-import org.elasticsearch.action.bulk.BulkItemResponse;
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase;
+import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction;
+import org.elasticsearch.action.ActionRequest;
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.transport.TransportClient;
-import org.elasticsearch.common.network.NetworkModule;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
-import org.elasticsearch.common.transport.TransportAddress;
-import org.elasticsearch.common.unit.ByteSizeUnit;
-import org.elasticsearch.common.unit.ByteSizeValue;
-import org.elasticsearch.common.unit.TimeValue;
-import org.elasticsearch.transport.Netty3Plugin;
-import org.elasticsearch.transport.client.PreBuiltTransportClient;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import java.net.InetSocketAddress;
-import java.util.ArrayList;
import java.util.List;
import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
/**
- * Sink that emits its input elements in bulk to an Elasticsearch cluster.
- *
+ * Elasticsearch 5.x sink that requests multiple {@link ActionRequest ActionRequests}
+ * against a cluster for each incoming element.
+ *
*
- * The first {@link Map} passed to the constructor is forwarded to Elasticsearch when creating
- * {@link TransportClient}. The config keys can be found in the Elasticsearch
- * documentation. An important setting is {@code cluster.name}, this should be set to the name
- * of the cluster that the sink should emit to.
+ * The sink internally uses a {@link TransportClient} 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.
+ *
*
- * Attention: When using the {@code TransportClient} the sink will fail if no cluster
- * can be connected to.
+ * 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.
+ *
*
- * The second {@link Map} is used to configure a {@link BulkProcessor} to send {@link IndexRequest IndexRequests}.
+ * 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 RequestIndexer}. This is used to create an
- * {@link IndexRequest} from an element that needs to be added to Elasticsearch. See
- * {@link RequestIndexer} for an example.
+ * 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 emitted by this sink
+ * @param Type of the elements handled by this sink
*/
-public class ElasticsearchSink 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 class ElasticsearchSink extends ElasticsearchSinkBase {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSink.class);
-
- /**
- * The user specified config map that we forward to Elasticsearch when we create the Client.
- */
- private final Map esConfig;
-
- /**
- * The user specified config map that we use to configure BulkProcessor.
- */
- private final Map sinkConfig;
-
- /**
- * The list of nodes that the TransportClient should connect to. This is null if we are using
- * an embedded Node to get a Client.
- */
- private final List transportAddresses;
-
- /**
- * The builder that is used to construct an {@link IndexRequest} from the incoming element.
- */
- private final ElasticsearchSinkFunction elasticsearchSinkFunction;
-
- /**
- * The Client that was either retrieved from a Node or is a TransportClient.
- */
- private transient Client client;
-
- /**
- * Bulk processor that was created using the client
- */
- private transient BulkProcessor bulkProcessor;
-
- /**
- * Bulk {@link org.elasticsearch.action.ActionRequest} indexer
- */
- private transient RequestIndexer requestIndexer;
-
- /**
- * This is set from inside the BulkProcessor listener if there where failures in processing.
- */
- private final AtomicBoolean hasFailure = new AtomicBoolean(false);
-
- /**
- * This is set from inside the BulkProcessor listener if a Throwable was thrown during processing.
- */
- private final AtomicReference failureThrowable = new AtomicReference<>();
-
/**
- * Creates a new ElasticsearchSink that connects to the cluster using a TransportClient.
+ * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link TransportClient}.
*
- * @param esConfig The map of user settings that are passed when constructing the TransportClient
- * @param sinkConfig The map of user settings that are passed when constructing the BulkProcessor
- * @param transportAddresses The Elasticsearch Nodes to which to connect using a {@code TransportClient}
- * @param elasticsearchSinkFunction This is used to generate the ActionRequest from the incoming element
+ * @param userConfig The map of user settings that are used when constructing the {@link TransportClient}
+ * @param transportAddresses The addresses of Elasticsearch nodes to which to connect using a {@link TransportClient}
+ * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element
*/
- public ElasticsearchSink(Map esConfig, Map sinkConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) {
- this.esConfig = esConfig;
- this.sinkConfig = sinkConfig;
- this.elasticsearchSinkFunction = elasticsearchSinkFunction;
- Preconditions.checkArgument(transportAddresses != null && transportAddresses.size() > 0);
- this.transportAddresses = transportAddresses;
+ public ElasticsearchSink(Map userConfig,
+ List transportAddresses,
+ ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ super(new Elasticsearch5ApiCallBridge(transportAddresses), userConfig, elasticsearchSinkFunction);
}
-
- /**
- * Initializes the connection to Elasticsearch by creating a
- * {@link org.elasticsearch.client.transport.TransportClient}.
- */
- @Override
- public void open(Configuration configuration) {
- List transportNodes;
- transportNodes = new ArrayList<>(transportAddresses.size());
- for (InetSocketAddress address : transportAddresses) {
- transportNodes.add(new InetSocketTransportAddress(address));
- }
-
- Settings settings = Settings.builder().put(esConfig)
- .put(NetworkModule.HTTP_TYPE_KEY, Netty3Plugin.NETTY_HTTP_TRANSPORT_NAME)
- .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty3Plugin.NETTY_TRANSPORT_NAME)
- .build();
-
- TransportClient transportClient = new PreBuiltTransportClient(settings);
- for (TransportAddress transport : transportNodes) {
- transportClient.addTransportAddress(transport);
- }
-
- // verify that we actually are connected to a cluster
- if (transportClient.connectedNodes().isEmpty()) {
- throw new RuntimeException("Client is not connected to any Elasticsearch nodes!");
- }
-
- client = transportClient;
-
- if (LOG.isInfoEnabled()) {
- LOG.info("Created Elasticsearch TransportClient {}", client);
- }
-
- 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()) {
- if (itemResp.isFailed()) {
- LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage());
- failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage()));
- }
- }
- hasFailure.set(true);
- }
- }
-
- @Override
- public void afterBulk(long executionId, BulkRequest request, Throwable failure) {
- LOG.error(failure.getMessage());
- failureThrowable.compareAndSet(null, failure);
- hasFailure.set(true);
- }
- });
-
- // This makes flush() blocking
- bulkProcessorBuilder.setConcurrentRequests(0);
-
- ParameterTool params = ParameterTool.fromMap(sinkConfig);
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS)) {
- bulkProcessorBuilder.setBulkActions(params.getInt(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB)) {
- bulkProcessorBuilder.setBulkSize(new ByteSizeValue(params.getInt(
- CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB), ByteSizeUnit.MB));
- }
-
- if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) {
- bulkProcessorBuilder.setFlushInterval(TimeValue.timeValueMillis(params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)));
- }
-
- bulkProcessor = bulkProcessorBuilder.build();
- requestIndexer = new BulkProcessorIndexer(bulkProcessor);
- }
-
- @Override
- public void invoke(T element) {
- elasticsearchSinkFunction.process(element, getRuntimeContext(), requestIndexer);
- }
-
- @Override
- public void close() {
- if (bulkProcessor != null) {
- bulkProcessor.close();
- bulkProcessor = null;
- }
-
- if (client != null) {
- client.close();
- }
-
- if (hasFailure.get()) {
- Throwable cause = failureThrowable.get();
- if (cause != null) {
- throw new RuntimeException("An error occurred in ElasticsearchSink.", cause);
- } else {
- throw new RuntimeException("An error occurred in ElasticsearchSink.");
- }
- }
-
- }
-
}
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/RequestIndexer.java
deleted file mode 100644
index 170df31cdacaf..0000000000000
--- a/flink-connectors/flink-connector-elasticsearch5/src/main/java/org/apache/flink/streaming/connectors/elasticsearch5/RequestIndexer.java
+++ /dev/null
@@ -1,25 +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.elasticsearch5;
-
-import org.elasticsearch.action.ActionRequest;
-
-import java.io.Serializable;
-
-public interface RequestIndexer extends Serializable {
- void add(ActionRequest... actionRequests);
-}
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java
new file mode 100644
index 0000000000000..f3d8897e93823
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.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.elasticsearch;
+
+import org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSinkITCase;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.network.NetworkModule;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.node.Node;
+import org.elasticsearch.node.internal.InternalSettingsPreparer;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.transport.Netty3Plugin;
+
+import java.io.File;
+import java.util.Collections;
+
+/**
+ * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 5.x.
+ * 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, Netty3Plugin.NETTY_HTTP_TRANSPORT_NAME)
+ .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty3Plugin.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(Netty3Plugin.class));
+ }
+ }
+
+}
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkITCase.java
index b4a370bdc1aec..3ebda523f5276 100644
--- a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkITCase.java
+++ b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/ElasticsearchSinkITCase.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -6,9 +6,9 @@
* 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,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,184 +17,54 @@
*/
package org.apache.flink.streaming.connectors.elasticsearch5;
-import com.google.common.collect.ImmutableMap;
-import org.apache.flink.api.common.functions.RuntimeContext;
-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.api.functions.source.SourceFunction;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.elasticsearch.action.get.GetResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.client.Requests;
-import org.elasticsearch.common.network.NetworkModule;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.node.Node;
-import org.elasticsearch.node.internal.InternalSettingsPreparer;
-import org.elasticsearch.plugins.Plugin;
-import org.elasticsearch.transport.Netty3Plugin;
-import org.junit.ClassRule;
+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.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import java.io.File;
import java.net.InetAddress;
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.assertEquals;
-import static org.junit.Assert.fail;
-
-public class ElasticsearchSinkITCase extends StreamingMultipleProgramsTestBase {
-
- private static final int NUM_ELEMENTS = 20;
-
- @ClassRule
- public static TemporaryFolder tempFolder = new TemporaryFolder();
+public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase {
@Test
public void testTransportClient() throws Exception {
-
- File dataDir = tempFolder.newFolder();
-
- Settings settings = Settings.builder()
- .put("cluster.name", "my-transport-client-cluster")
- .put("http.enabled", false)
- .put("path.home", dataDir.getParent())
- .put("path.data", dataDir.getAbsolutePath())
- .put(NetworkModule.HTTP_TYPE_KEY, Netty3Plugin.NETTY_HTTP_TRANSPORT_NAME)
- .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty3Plugin.NETTY_TRANSPORT_NAME)
- .build();
-
- Node node = new PluginNode(settings);
- node.start();
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map esConfig = ImmutableMap.of("cluster.name", "my-transport-client-cluster");
- Map sinkConfig = ImmutableMap.of(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
- List transports = new ArrayList<>();
- transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
- source.addSink(new ElasticsearchSink<>(esConfig, sinkConfig, transports, new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch TransportClient Test");
-
- // verify the results
- Client client = node.client();
- for (int i = 0; i < NUM_ELEMENTS; i++) {
- GetResponse response = client.prepareGet("my-index", "my-type", Integer.toString(i)).get();
- assertEquals("message #" + i, response.getSource().get("data"));
- }
-
- node.close();
+ runTransportClientTest();
}
- @Test(expected = IllegalArgumentException.class)
+ @Test
public void testNullTransportClient() throws Exception {
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map esConfig = ImmutableMap.of("cluster.name", "my-transport-client-cluster");
- Map sinkConfig = ImmutableMap.of(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
- source.addSink(new ElasticsearchSink<>(esConfig, sinkConfig, null, new TestElasticsearchSinkFunction()));
-
- fail();
+ runNullTransportClientTest();
}
- @Test(expected = IllegalArgumentException.class)
+ @Test
public void testEmptyTransportClient() throws Exception {
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map esConfig = ImmutableMap.of("cluster.name", "my-transport-client-cluster");
- Map sinkConfig = ImmutableMap.of(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
- source.addSink(new ElasticsearchSink<>(esConfig, sinkConfig, new ArrayList(), new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch TransportClient Test");
-
- fail();
+ runEmptyTransportClientTest();
}
- @Test(expected = JobExecutionException.class)
+ @Test
public void testTransportClientFails() throws Exception {
- // this checks whether the TransportClient fails early when there is no cluster to
- // connect to. There isn't a similar test for the Node Client version since that
- // one will block and wait for a cluster to come online
-
- final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-
- DataStreamSource> source = env.addSource(new TestSourceFunction());
-
- Map esConfig = ImmutableMap.of("cluster.name", "my-transport-client-cluster");
- Map sinkConfig = ImmutableMap.of(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
-
- List transports = new ArrayList<>();
- transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
-
- source.addSink(new ElasticsearchSink<>(esConfig, sinkConfig, transports, new TestElasticsearchSinkFunction()));
-
- env.execute("Elasticsearch Node Client Test");
-
- fail();
+ runTransportClientFailsTest();
}
- private static class TestSourceFunction implements SourceFunction> {
- private static final long serialVersionUID = 1L;
-
- private volatile boolean running = true;
-
- @Override
- public void run(SourceContext> ctx) throws Exception {
- for (int i = 0; i < NUM_ELEMENTS && running; i++) {
- ctx.collect(Tuple2.of(i, "message #" + i));
- }
- }
-
- @Override
- public void cancel() {
- running = false;
- }
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig,
+ List transportAddresses,
+ ElasticsearchSinkFunction elasticsearchSinkFunction) {
+ return new ElasticsearchSink<>(userConfig, transportAddresses, elasticsearchSinkFunction);
}
- private static class TestElasticsearchSinkFunction implements ElasticsearchSinkFunction> {
- private static final long serialVersionUID = 1L;
-
- public IndexRequest createIndexRequest(Tuple2 element) {
- Map json = new HashMap<>();
- json.put("data", element.f1);
+ @Override
+ protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(
+ Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception {
- return Requests.indexRequest()
- .index("my-index")
- .type("my-type")
- .id(element.f0.toString())
- .source(json);
- }
+ List transports = new ArrayList<>();
+ transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
- @Override
- public void process(Tuple2 element, RuntimeContext ctx, RequestIndexer indexer) {
- indexer.add(createIndexRequest(element));
- }
+ return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction);
}
- private static class PluginNode extends Node {
- public PluginNode(Settings settings) {
- super(InternalSettingsPreparer.prepareEnvironment(settings, null), Collections.>singletonList(Netty3Plugin.class));
- }
- }
}
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchExample.java b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchSinkExample.java
similarity index 72%
rename from flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchExample.java
rename to flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchSinkExample.java
index 47ce846338f01..4135283420972 100644
--- a/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchExample.java
+++ b/flink-connectors/flink-connector-elasticsearch5/src/test/java/org/apache/flink/streaming/connectors/elasticsearch5/examples/ElasticsearchSinkExample.java
@@ -16,14 +16,13 @@
*/
package org.apache.flink.streaming.connectors.elasticsearch5.examples;
-import com.google.common.collect.ImmutableMap;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+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.elasticsearch5.ElasticsearchSink;
-import org.apache.flink.streaming.connectors.elasticsearch5.ElasticsearchSinkFunction;
-import org.apache.flink.streaming.connectors.elasticsearch5.RequestIndexer;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Requests;
@@ -38,36 +37,35 @@
* 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 ElasticsearchExample {
+public class ElasticsearchSinkExample {
public static void main(String[] args) throws Exception {
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- SingleOutputStreamOperator source =
- env.generateSequence(0, 20).map(new MapFunction() {
- @Override
- public String map(Long value) throws Exception {
- return "message #" + value;
- }
- });
-
- Map esConfig = ImmutableMap.of("cluster.name", "elasticsearch");
+ DataStream source = env.generateSequence(0, 20).map(new MapFunction() {
+ @Override
+ public String map(Long value) throws Exception {
+ return "message #" + value;
+ }
+ });
+ Map userConfig = new HashMap<>();
+ userConfig.put("cluster.name", "elasticsearch");
// This instructs the sink to emit after every element, otherwise they would be buffered
- Map sinkConfig = ImmutableMap.of(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
+ userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1");
List transports = new ArrayList<>();
transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300));
- source.addSink(new ElasticsearchSink<>(esConfig, sinkConfig, transports, new ElasticsearchSinkFunction() {
+ source.addSink(new ElasticsearchSink<>(userConfig, transports, new ElasticsearchSinkFunction() {
@Override
public void process(String element, RuntimeContext ctx, RequestIndexer indexer) {
indexer.add(createIndexRequest(element));
}
}));
- env.execute("Elasticsearch Example");
+ env.execute("Elasticsearch Sink Example");
}
private static IndexRequest createIndexRequest(String element) {
diff --git a/flink-connectors/flink-connector-elasticsearch5/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch5/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000000000..20551848eea24
--- /dev/null
+++ b/flink-connectors/flink-connector-elasticsearch5/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/pom.xml b/flink-connectors/pom.xml
index e19c77f5a72f7..5d8ca700f951d 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -45,9 +45,9 @@ under the License.
flink-connector-kafka-0.8
flink-connector-kafka-0.9
flink-connector-kafka-0.10
+ flink-connector-elasticsearch-base
flink-connector-elasticsearch
flink-connector-elasticsearch2
- flink-connector-elasticsearch5
flink-connector-rabbitmq
flink-connector-twitter
flink-connector-nifi
@@ -86,6 +86,20 @@ under the License.
flink-connector-kinesis
+
+
+
+ include-elasticsearch5
+
+ 1.8
+
+
+ flink-connector-elasticsearch5
+
+