From 8640a4cc6a2400b58b8e44bc9341fa075c4462b6 Mon Sep 17 00:00:00 2001 From: Christophe Jolif Date: Fri, 18 May 2018 00:17:04 +0200 Subject: [PATCH 01/21] [FLINK-7386] [elasticsearch] Evolve ES connector API to make it working with Elasticsearch 5.3+ This closes #6043. --- .../elasticsearch/BulkProcessorIndexer.java | 29 +++++++++++-- .../ElasticsearchApiCallBridge.java | 18 ++++---- .../elasticsearch/ElasticsearchSinkBase.java | 4 +- .../elasticsearch/RequestIndexer.java | 42 ++++++++++++++++++- .../ElasticsearchSinkBaseTest.java | 37 ++++++++-------- .../Elasticsearch1ApiCallBridge.java | 9 +++- .../Elasticsearch2ApiCallBridge.java | 15 ++++--- .../Elasticsearch5ApiCallBridge.java | 15 ++++--- 8 files changed, 119 insertions(+), 50 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java index 2ebb97c82e213..33b42cb47f184 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/BulkProcessorIndexer.java @@ -22,6 +22,9 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; import java.util.concurrent.atomic.AtomicLong; @@ -45,12 +48,32 @@ class BulkProcessorIndexer implements RequestIndexer { } @Override - public void add(ActionRequest... actionRequests) { - for (ActionRequest actionRequest : actionRequests) { + public void add(DeleteRequest... deleteRequests) { + for (DeleteRequest deleteRequest : deleteRequests) { if (flushOnCheckpoint) { numPendingRequestsRef.getAndIncrement(); } - this.bulkProcessor.add(actionRequest); + this.bulkProcessor.add(deleteRequest); + } + } + + @Override + public void add(IndexRequest... indexRequests) { + for (IndexRequest indexRequest : indexRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(indexRequest); + } + } + + @Override + public void add(UpdateRequest... updateRequests) { + for (UpdateRequest updateRequest : updateRequests) { + if (flushOnCheckpoint) { + numPendingRequestsRef.getAndIncrement(); + } + this.bulkProcessor.add(updateRequest); } } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 2a7a21659e406..1c501bf4a209d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -22,7 +22,6 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.client.Client; import javax.annotation.Nullable; @@ -39,15 +38,18 @@ * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. */ @Internal -public interface ElasticsearchApiCallBridge extends Serializable { +public abstract class ElasticsearchApiCallBridge implements Serializable { /** - * Creates an Elasticsearch {@link Client}. + * Creates an Elasticsearch client implementing {@link AutoCloseable}. This can + * be a {@link org.elasticsearch.client.Client} or {@link org.elasticsearch.client.RestHighLevelClient} * * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - Client createClient(Map clientConfig); + public abstract AutoCloseable createClient(Map clientConfig); + + public abstract BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener); /** * Extracts the cause of failure of a bulk item action. @@ -55,7 +57,7 @@ public interface ElasticsearchApiCallBridge extends Serializable { * @param bulkItemResponse the bulk item response to extract cause of failure * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). */ - @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + public abstract @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); /** * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. @@ -64,13 +66,15 @@ public interface ElasticsearchApiCallBridge extends Serializable { * @param builder the {@link BulkProcessor.Builder} to configure. * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). */ - void configureBulkProcessorBackoff( + public abstract void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); /** * Perform any necessary state cleanup. */ - void cleanup(); + public void cleanup() { + // nothing to cleanup by default + } } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 9105d9947f204..0305ee3d86766 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -176,7 +176,7 @@ public void setDelayMillis(long delayMillis) { private AtomicLong numPendingRequests = new AtomicLong(0); /** Elasticsearch client created using the call bridge. */ - private transient Client client; + private transient AutoCloseable client; /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ private transient BulkProcessor bulkProcessor; @@ -341,7 +341,7 @@ public void close() throws Exception { protected BulkProcessor buildBulkProcessor(BulkProcessor.Listener listener) { checkNotNull(listener); - BulkProcessor.Builder bulkProcessorBuilder = BulkProcessor.builder(client, listener); + BulkProcessor.Builder bulkProcessorBuilder = callBridge.createBulkProcessorBuilder(client, listener); // This makes flush() blocking bulkProcessorBuilder.setConcurrentRequests(0); diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java index 2a1b29736b69b..3dc8f87964197 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/RequestIndexer.java @@ -21,9 +21,12 @@ import org.apache.flink.annotation.PublicEvolving; import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; /** - * Users add multiple {@link ActionRequest ActionRequests} to a {@link RequestIndexer} to prepare + * Users add multiple delete, index or update requests to a {@link RequestIndexer} to prepare * them for sending to an Elasticsearch cluster. */ @PublicEvolving @@ -33,6 +36,41 @@ public interface RequestIndexer { * Add multiple {@link ActionRequest} to the indexer to prepare for sending requests to Elasticsearch. * * @param actionRequests The multiple {@link ActionRequest} to add. + * @deprecated use the {@link DeleteRequest}, {@link IndexRequest} or {@link UpdateRequest} */ - void add(ActionRequest... actionRequests); + @Deprecated + default void add(ActionRequest... actionRequests) { + for (ActionRequest actionRequest : actionRequests) { + if (actionRequest instanceof IndexRequest) { + add((IndexRequest) actionRequest); + } else if (actionRequest instanceof DeleteRequest) { + add((DeleteRequest) actionRequest); + } else if (actionRequest instanceof UpdateRequest) { + add((UpdateRequest) actionRequest); + } else { + throw new IllegalArgumentException("RequestIndexer only supports Index, Delete and Update requests"); + } + } + } + + /** + * Add multiple {@link DeleteRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param deleteRequests The multiple {@link DeleteRequest} to add. + */ + void add(DeleteRequest... deleteRequests); + + /** + * Add multiple {@link IndexRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param indexRequests The multiple {@link IndexRequest} to add. + */ + void add(IndexRequest... indexRequests); + + /** + * Add multiple {@link UpdateRequest} to the indexer to prepare for sending requests to Elasticsearch. + * + * @param updateRequests The multiple {@link UpdateRequest} to add. + */ + void add(UpdateRequest... updateRequests); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 09d8806b963ee..5a161a747c13b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; import org.junit.Assert; @@ -92,7 +93,7 @@ public void testItemFailureRethrownOnInvoke() throws Throwable { // setup the next bulk request, and its mock item failures sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -124,7 +125,7 @@ public void testItemFailureRethrownOnCheckpoint() throws Throwable { // setup the next bulk request, and its mock item failures sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -164,7 +165,7 @@ public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { sink.setMockItemFailuresListForNextBulkItemResponses(mockResponsesList); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request (1 request only, thus should succeed) sink.manualBulkRequestWithAllPendingRequests(); @@ -172,7 +173,7 @@ public void testItemFailureRethrownOnCheckpointAfterFlush() throws Throwable { // setup the requests to be flushed in the snapshot testHarness.processElement(new StreamRecord<>("msg-2")); testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -217,7 +218,7 @@ public void testBulkFailureRethrownOnInvoke() throws Throwable { // setup the next bulk request, and let the whole bulk request fail sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -249,7 +250,7 @@ public void testBulkFailureRethrownOnCheckpoint() throws Throwable { // setup the next bulk request, and let the whole bulk request fail sink.setFailNextBulkRequestCompletely(new Exception("artificial failure for bulk request")); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -284,7 +285,7 @@ public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { // setup the next bulk request, and let bulk request succeed sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList((Exception) null)); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // manually execute the next bulk request sink.manualBulkRequestWithAllPendingRequests(); @@ -292,7 +293,7 @@ public void testBulkFailureRethrownOnOnCheckpointAfterFlush() throws Throwable { // setup the requests to be flushed in the snapshot testHarness.processElement(new StreamRecord<>("msg-2")); testHarness.processElement(new StreamRecord<>("msg-3")); - verify(sink.getMockBulkProcessor(), times(3)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(3)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -346,7 +347,7 @@ public void testAtLeastOnceSink() throws Throwable { // it contains 1 request, which will fail and re-added to the next bulk request sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); CheckedThread snapshotThread = new CheckedThread() { @Override @@ -402,7 +403,7 @@ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Excepti // setup the next bulk request, and let bulk request succeed sink.setMockItemFailuresListForNextBulkItemResponses(Collections.singletonList(new Exception("artificial failure for record"))); testHarness.processElement(new StreamRecord<>("msg-1")); - verify(sink.getMockBulkProcessor(), times(1)).add(any(ActionRequest.class)); + verify(sink.getMockBulkProcessor(), times(1)).add(any(IndexRequest.class)); // the snapshot should not block even though we haven't flushed the bulk request testHarness.snapshot(1L, 1000L); @@ -478,11 +479,11 @@ public BulkProcessor getMockBulkProcessor() { protected BulkProcessor buildBulkProcessor(final BulkProcessor.Listener listener) { this.mockBulkProcessor = mock(BulkProcessor.class); - when(mockBulkProcessor.add(any(ActionRequest.class))).thenAnswer(new Answer() { + when(mockBulkProcessor.add(any(IndexRequest.class))).thenAnswer(new Answer() { @Override public Object answer(InvocationOnMock invocationOnMock) throws Throwable { // intercept the request and add it to our mock bulk request - nextBulkRequest.add(invocationOnMock.getArgumentAt(0, ActionRequest.class)); + nextBulkRequest.add(invocationOnMock.getArgumentAt(0, IndexRequest.class)); return null; } @@ -530,12 +531,12 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { } } - private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { + private static class DummyElasticsearchApiCallBridge extends ElasticsearchApiCallBridge { private static final long serialVersionUID = -4272760730959041699L; @Override - public Client createClient(Map clientConfig) { + public AutoCloseable createClient(Map clientConfig) { return mock(Client.class); } @@ -550,13 +551,13 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } @Override - public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { - // no need for this in the test cases here + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return null; } @Override - public void cleanup() { - // nothing to cleanup + public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + // no need for this in the test cases here } } 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 index 2a3c2a0646088..6f492064a02d3 100644 --- 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 @@ -42,7 +42,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 1.x. */ @Internal -public class Elasticsearch1ApiCallBridge implements ElasticsearchApiCallBridge { +public class Elasticsearch1ApiCallBridge extends ElasticsearchApiCallBridge { private static final long serialVersionUID = -2632363720584123682L; @@ -70,7 +70,7 @@ public class Elasticsearch1ApiCallBridge implements ElasticsearchApiCallBridge { } @Override - public Client createClient(Map clientConfig) { + public AutoCloseable createClient(Map clientConfig) { if (transportAddresses == null) { // Make sure that we disable http access to our embedded node @@ -115,6 +115,11 @@ public Client createClient(Map clientConfig) { } } + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return BulkProcessor.builder((Client) client, listener); + } + @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { if (!bulkItemResponse.isFailed()) { 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 index 390a4078e2b58..80c1b3acf2204 100644 --- 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 @@ -44,7 +44,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 2.x. */ @Internal -public class Elasticsearch2ApiCallBridge implements ElasticsearchApiCallBridge { +public class Elasticsearch2ApiCallBridge extends ElasticsearchApiCallBridge { private static final long serialVersionUID = 2638252694744361079L; @@ -63,7 +63,7 @@ public class Elasticsearch2ApiCallBridge implements ElasticsearchApiCallBridge { } @Override - public Client createClient(Map clientConfig) { + public AutoCloseable createClient(Map clientConfig) { Settings settings = Settings.settingsBuilder().put(clientConfig).build(); TransportClient transportClient = TransportClient.builder().settings(settings).build(); @@ -83,6 +83,11 @@ public Client createClient(Map clientConfig) { return transportClient; } + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return BulkProcessor.builder((Client) client, listener); + } + @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { if (!bulkItemResponse.isFailed()) { @@ -117,10 +122,4 @@ public void configureBulkProcessorBackoff( builder.setBackoffPolicy(backoffPolicy); } - - @Override - public void cleanup() { - // nothing to cleanup - } - } 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 index 7c4ba7a97f1dd..1e73feb9e4377 100644 --- 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 @@ -47,7 +47,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 5.x. */ @Internal -public class Elasticsearch5ApiCallBridge implements ElasticsearchApiCallBridge { +public class Elasticsearch5ApiCallBridge extends ElasticsearchApiCallBridge { private static final long serialVersionUID = -5222683870097809633L; @@ -66,7 +66,7 @@ public class Elasticsearch5ApiCallBridge implements ElasticsearchApiCallBridge { } @Override - public Client createClient(Map clientConfig) { + public AutoCloseable 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) @@ -89,6 +89,11 @@ public Client createClient(Map clientConfig) { return transportClient; } + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return BulkProcessor.builder((Client) client, listener); + } + @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { if (!bulkItemResponse.isFailed()) { @@ -123,10 +128,4 @@ public void configureBulkProcessorBackoff( builder.setBackoffPolicy(backoffPolicy); } - - @Override - public void cleanup() { - // nothing to cleanup - } - } From db6b5c1f6516d97d19e8df545610fabdfcff450e Mon Sep 17 00:00:00 2001 From: Christophe Jolif Date: Thu, 25 Jan 2018 22:31:57 +0100 Subject: [PATCH 02/21] [FLINK-8101] [elasticsearch] Elasticsearch 6.X REST support --- docs/dev/connectors/elasticsearch.md | 55 +++++- .../ElasticsearchApiCallBridge.java | 3 +- .../elasticsearch/ElasticsearchSinkBase.java | 3 +- .../ElasticsearchSinkBaseTest.java | 10 +- .../EmbeddedElasticsearchNodeEnvironment.java | 2 +- .../Elasticsearch1ApiCallBridge.java | 2 +- .../flink-connector-elasticsearch6/pom.xml | 180 ++++++++++++++++++ .../Elasticsearch6ApiCallBridge.java | 110 +++++++++++ .../elasticsearch6/ElasticsearchSink.java | 91 +++++++++ ...eddedElasticsearchNodeEnvironmentImpl.java | 82 ++++++++ .../ElasticsearchSinkITCase.java | 152 +++++++++++++++ .../examples/ElasticsearchSinkExample.java | 81 ++++++++ .../src/test/resources/log4j-test.properties | 27 +++ flink-connectors/pom.xml | 1 + 14 files changed, 786 insertions(+), 13 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch6/pom.xml create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 52d1b58bf514d..20a7d7138ee51 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -55,6 +55,11 @@ of the Elasticsearch installation: 1.3.0 5.x + + flink-connector-elasticsearch6{{ site.scala_version_suffix }} + 1.6.0 + 6 and later versions + @@ -71,7 +76,7 @@ creating an `ElasticsearchSink` for requesting document actions against your clu ## Elasticsearch Sink -The `ElasticsearchSink` uses a `TransportClient` to communicate with an +The `ElasticsearchSink` uses a `TransportClient` (before 6.x) or `RestHighLevelClient` (starting with 6.x) to communicate with an Elasticsearch cluster. The example below shows how to configure and create a sink: @@ -138,6 +143,31 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea } }));{% endhighlight %} +
+{% highlight java %} +DataStream input = ...; + +List httpHost = new ArrayList<>(); +httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); +httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")); + +input.addSink(new ElasticsearchSink<>(httpHosts, new ElasticsearchSinkFunction() { + public IndexRequest createIndexRequest(String element) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json); + } + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } +}));{% endhighlight %} +
{% highlight scala %} val input: DataStream[String] = ... @@ -190,9 +220,30 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc })) {% endhighlight %}
+
+{% highlight scala %} +val input: DataStream[String] = ... + +val httpHosts = new java.util.ArrayList[HttpHost] +httpHosts.add(new HttpHost("127.0.0.1", 9300, "http")) +httpHosts.add(new HttpHost("10.2.3.1", 9300, "http")) + +input.addSink(new ElasticsearchSink(httpHosts, new ElasticsearchSinkFunction[String] { + def createIndexRequest(element: String): IndexRequest = { + val json = new java.util.HashMap[String, String] + json.put("data", element) + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json) + } +})) +{% endhighlight %} +
-Note how a `Map` of `String`s is used to configure the `ElasticsearchSink`. +Note how `TransportClient` based version use a `Map` of `String`s is used to configure the `ElasticsearchSink`. 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 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 1c501bf4a209d..90d84f3635a10 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -41,8 +41,7 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { /** - * Creates an Elasticsearch client implementing {@link AutoCloseable}. This can - * be a {@link org.elasticsearch.client.Client} or {@link org.elasticsearch.client.RestHighLevelClient} + * Creates an Elasticsearch client implementing {@link AutoCloseable}. * * @param clientConfig The configuration to use when constructing the client. * @return The created client. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 0305ee3d86766..98304844410b0 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -32,7 +32,6 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -59,7 +58,7 @@ * *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, - * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. + * for example, to create a Elasticsearch {@link Client} or {@RestHighLevelClient}, handle failed item responses, etc. * * @param Type of the elements handled by this sink */ diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 5a161a747c13b..460e939fab9db 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -540,6 +540,11 @@ public AutoCloseable createClient(Map clientConfig) { return mock(Client.class); } + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + return null; + } + @Nullable @Override public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { @@ -550,11 +555,6 @@ public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkIt } } - @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - return null; - } - @Override public void configureBulkProcessorBackoff(BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { // no need for this in the test cases here diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java index ea6e7a3ac70b1..fd14ba3637071 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironment.java @@ -29,7 +29,7 @@ * also be located under the same package. The intentional package-private accessibility of this interface * enforces that. */ -interface EmbeddedElasticsearchNodeEnvironment { +public interface EmbeddedElasticsearchNodeEnvironment { /** * Start an embedded Elasticsearch node instance. diff --git a/flink-connectors/flink-connector-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 index 6f492064a02d3..28d5f345ff007 100644 --- 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 @@ -85,7 +85,7 @@ public AutoCloseable createClient(Map clientConfig) { .data(false) .node(); - Client client = node.client(); + AutoCloseable client = node.client(); if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch client from embedded node"); diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml new file mode 100644 index 0000000000000..e453837fbc996 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -0,0 +1,180 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-connectors + 1.7-SNAPSHOT + .. + + + flink-connector-elasticsearch6_${scala.binary.version} + flink-connector-elasticsearch6 + + jar + + + + 6.3.1 + + + + + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + + org.elasticsearch + elasticsearch + + + + + + + org.elasticsearch.client + elasticsearch-rest-high-level-client + ${elasticsearch.version} + + + + + + org.apache.logging.log4j + log4j-to-slf4j + 2.7 + + + + + + org.apache.flink + flink-test-utils_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + test + test-jar + + + + org.apache.flink + flink-connector-elasticsearch-base_${scala.binary.version} + ${project.version} + + + org.elasticsearch + elasticsearch + + + test-jar + test + + + + + + org.elasticsearch.client + transport + ${elasticsearch.version} + test + + + + org.elasticsearch.plugin + transport-netty4-client + ${elasticsearch.version} + test + + + + + + org.apache.logging.log4j + log4j-api + 2.7 + test + + + + org.apache.logging.log4j + log4j-core + 2.7 + test + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.2 + + + org.apache.logging.log4j:log4j-to-slf4j + + + + + + + diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java new file mode 100644 index 0000000000000..2cb4ea03a13f8 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch6; + +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.util.Preconditions; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.bulk.BackoffPolicy; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestHighLevelClient; +import org.elasticsearch.common.unit.TimeValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. + */ +public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { + + private static final long serialVersionUID = -5222683870097809633L; + + private static final Logger LOG = LoggerFactory.getLogger(Elasticsearch6ApiCallBridge.class); + + /** + * User-provided HTTP Host. + */ + private final List httpHosts; + + Elasticsearch6ApiCallBridge(List httpHosts) { + Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); + this.httpHosts = httpHosts; + } + + @Override + public AutoCloseable createClient(Map clientConfig) { + RestHighLevelClient rhlClient = + new RestHighLevelClient(RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]))); + + if (LOG.isInfoEnabled()) { + LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); + } + + return rhlClient; + } + + @Override + public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + RestHighLevelClient rhlClient = (RestHighLevelClient) client; + return BulkProcessor.builder(rhlClient::bulkAsync, listener); + } + + @Override + public Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse) { + if (!bulkItemResponse.isFailed()) { + return null; + } else { + return bulkItemResponse.getFailure().getCause(); + } + } + + @Override + public void configureBulkProcessorBackoff( + BulkProcessor.Builder builder, + @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy) { + + BackoffPolicy backoffPolicy; + if (flushBackoffPolicy != null) { + switch (flushBackoffPolicy.getBackoffType()) { + case CONSTANT: + backoffPolicy = BackoffPolicy.constantBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + break; + case EXPONENTIAL: + default: + backoffPolicy = BackoffPolicy.exponentialBackoff( + new TimeValue(flushBackoffPolicy.getDelayMillis()), + flushBackoffPolicy.getMaxRetryCount()); + } + } else { + backoffPolicy = BackoffPolicy.noBackoff(); + } + + builder.setBackoffPolicy(backoffPolicy); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java new file mode 100644 index 0000000000000..3f75b5ff190d7 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch6; + +import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.bulk.BulkProcessor; +import org.elasticsearch.client.RestHighLevelClient; + +import java.util.List; +import java.util.Map; + +/** + * Elasticsearch 6.x sink that requests multiple {@link ActionRequest ActionRequests} + * against a cluster for each incoming element. + * + *

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

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

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

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

You also have to provide an {@link ElasticsearchSinkFunction}. This is used to create multiple + * {@link ActionRequest ActionRequests} for each incoming element. See the class level documentation of + * {@link ElasticsearchSinkFunction} for an example. + * + * @param Type of the elements handled by this sink + */ +public class ElasticsearchSink extends ElasticsearchSinkBase { + + private static final long serialVersionUID = 1L; + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element + * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + */ + public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + + this(userConfig, httpHosts, elasticsearchSinkFunction, new NoOpFailureHandler()); + } + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element + * @param failureHandler This is used to handle failed {@link ActionRequest} + * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + */ + public ElasticsearchSink( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler) { + + super(new Elasticsearch6ApiCallBridge(httpHosts), userConfig, elasticsearchSinkFunction, failureHandler); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java new file mode 100644 index 0000000000000..f419b41b49663 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch; + +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase; + +import org.elasticsearch.client.Client; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.InternalSettingsPreparer; +import org.elasticsearch.node.Node; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.transport.Netty4Plugin; + +import java.io.File; +import java.util.Collections; + +/** + * Implementation of {@link EmbeddedElasticsearchNodeEnvironment} for Elasticsearch 6. + * Will be dynamically loaded in {@link ElasticsearchSinkITCase} for integration tests. + */ +public class EmbeddedElasticsearchNodeEnvironmentImpl implements EmbeddedElasticsearchNodeEnvironment { + + private Node node; + + @Override + public void start(File tmpDataFolder, String clusterName) throws Exception { + if (node == null) { + Settings settings = Settings.builder() + .put("cluster.name", clusterName) + .put("http.enabled", false) + .put("path.home", tmpDataFolder.getParent()) + .put("path.data", tmpDataFolder.getAbsolutePath()) + .put(NetworkModule.HTTP_TYPE_KEY, Netty4Plugin.NETTY_HTTP_TRANSPORT_NAME) + .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4Plugin.NETTY_TRANSPORT_NAME) + .build(); + + node = new PluginNode(settings); + node.start(); + } + } + + @Override + public void close() throws Exception { + if (node != null && !node.isClosed()) { + node.close(); + node = null; + } + } + + @Override + public Client getClient() { + if (node != null && !node.isClosed()) { + return node.client(); + } else { + return null; + } + } + + private static class PluginNode extends Node { + public PluginNode(Settings settings) { + super(InternalSettingsPreparer.prepareEnvironment(settings, null), Collections.>singletonList(Netty4Plugin.class)); + } + } + +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java new file mode 100644 index 0000000000000..217077171c63d --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch6; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.RestHighLevelClient; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * IT cases for the {@link ElasticsearchSink}. + */ +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { + + /** + * Tests that the Elasticsearch sink works properly using a {@link RestHighLevelClient}. + */ + public void runTransportClientTest() throws Exception { + final String index = "transport-client-test-index"; + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + + env.execute("Elasticsearch RestHighLevelClient Test"); + + // verify the results + Client client = embeddedNodeEnv.getClient(); + SourceSinkDataTestKit.verifyProducedSinkData(client, index); + + client.close(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. + */ + public void runNullTransportClientTest() throws Exception { + try { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + createElasticsearchSink6(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. + */ + public void runEmptyTransportClientTest() throws Exception { + try { + Map userConfig = new HashMap<>(); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + createElasticsearchSink6(userConfig, + Collections.emptyList(), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + } catch (IllegalArgumentException expectedException) { + // test passes + return; + } + + fail(); + } + + /** + * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. + */ + public void runTransportClientFailsTest() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + + try { + env.execute("Elasticsearch Transport Client Test"); + } catch (JobExecutionException expectedException) { + assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); + return; + } + + fail(); + } + + @Override + protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { + return null; + } + + @Override + protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + ArrayList httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + } + + private ElasticsearchSinkBase createElasticsearchSink6( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction) { + return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java new file mode 100644 index 0000000000000..de1670feac4f5 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch6.examples; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that + * you have a cluster named "elasticsearch" running or change the name of cluster in the config map. + */ +public class ElasticsearchSinkExample { + + public static void main(String[] args) throws Exception { + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + DataStream source = env.generateSequence(0, 20).map(new MapFunction() { + @Override + public String map(Long value) throws Exception { + return "message #" + value; + } + }); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + List httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + + source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } + })); + + env.execute("Elasticsearch Sink Example"); + } + + private static IndexRequest createIndexRequest(String element) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .id(element) + .source(json); + } +} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties new file mode 100644 index 0000000000000..20551848eea24 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties @@ -0,0 +1,27 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +log4j.rootLogger=INFO, testlogger + +log4j.appender.testlogger=org.apache.log4j.ConsoleAppender +log4j.appender.testlogger.target=System.err +log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout +log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +# suppress the irrelevant (wrong) warnings from the netty channel handler +log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index 3afb779a80ecf..cacea91578e7d 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -50,6 +50,7 @@ under the License. flink-connector-elasticsearch flink-connector-elasticsearch2 flink-connector-elasticsearch5 + flink-connector-elasticsearch6 flink-connector-rabbitmq flink-connector-twitter flink-connector-nifi From 5a4523a03ce5d5e9f9957a989c4159a4d10986b7 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 23 Jul 2018 17:51:31 +0800 Subject: [PATCH 03/21] [FLINK-9885] [tests] Add Elasticsearch 6.x end-to-end test --- .../flink-elasticsearch6-test/pom.xml | 92 +++++++++++++++++++ .../tests/Elasticsearch6SinkExample.java | 92 +++++++++++++++++++ flink-end-to-end-tests/pom.xml | 1 + flink-end-to-end-tests/run-nightly-tests.sh | 1 + 4 files changed, 186 insertions(+) create mode 100644 flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml create mode 100644 flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml new file mode 100644 index 0000000000000..d170235c7022f --- /dev/null +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml @@ -0,0 +1,92 @@ + + + + + 4.0.0 + + + org.apache.flink + flink-end-to-end-tests + 1.7-SNAPSHOT + .. + + + flink-elasticsearch6-test + flink-elasticsearch6-test + jar + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${project.version} + provided + + + org.apache.flink + flink-connector-elasticsearch6_${scala.binary.version} + ${project.version} + + + + + + + org.apache.maven.plugins + maven-shade-plugin + 3.0.0 + + + package + + shade + + + Elasticsearch6SinkExample + + + com.google.code.findbugs:jsr305 + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + org.apache.flink.streaming.tests.Elasticsearch6SinkExample + + + + + + + + + + diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java new file mode 100644 index 0000000000000..5544ea58f3e42 --- /dev/null +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.tests; + +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.java.utils.ParameterTool; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * End to end test for Elasticsearch6Sink. + */ +public class Elasticsearch6SinkExample { + + public static void main(String[] args) throws Exception { + + final ParameterTool parameterTool = ParameterTool.fromArgs(args); + + if (parameterTool.getNumberOfParameters() < 3) { + System.out.println("Missing parameters!\n" + + "Usage: --numRecords --index --type "); + return; + } + + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().disableSysoutLogging(); + env.enableCheckpointing(5000); + + DataStream source = env.generateSequence(0, parameterTool.getInt("numRecords") - 1) + .map(new MapFunction() { + @Override + public String map(Long value) throws Exception { + return "message #" + value; + } + }); + + Map userConfig = new HashMap<>(); + // This instructs the sink to emit after every element, otherwise they would be buffered + userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + + List httpHosts = new ArrayList<>(); + httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + + source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element, parameterTool)); + } + })); + + env.execute("Elasticsearch 6.x end to end sink test example"); + } + + private static IndexRequest createIndexRequest(String element, ParameterTool parameterTool) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index(parameterTool.getRequired("index")) + .type(parameterTool.getRequired("type")) + .id(element) + .source(json); + } +} diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 4abf59509dfa8..5752d9afceda5 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -48,6 +48,7 @@ under the License. flink-elasticsearch1-test flink-elasticsearch2-test flink-elasticsearch5-test + flink-elasticsearch6-test flink-quickstart-test flink-confluent-schema-registry flink-stream-state-ttl-test diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh index dc8424f25eccf..431c21ee7758f 100755 --- a/flink-end-to-end-tests/run-nightly-tests.sh +++ b/flink-end-to-end-tests/run-nightly-tests.sh @@ -96,6 +96,7 @@ run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR run_test "Elasticsearch (v1.7.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz" run_test "Elasticsearch (v2.3.5) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz" run_test "Elasticsearch (v5.1.2) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz" +run_test "Elasticsearch (v6.3.1) sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 6 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-6.3.1.tar.gz" run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java" run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala" From e1fb6e4a821d9ddc722cbf9bb26b75b1f427651c Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 25 Jul 2018 17:21:34 +0800 Subject: [PATCH 04/21] fixup! Address review comments --- docs/dev/connectors/elasticsearch.md | 2 +- .../ElasticsearchApiCallBridge.java | 21 ++- .../elasticsearch/ElasticsearchSinkBase.java | 7 +- .../ElasticsearchSinkBaseTest.java | 8 +- .../ElasticsearchSinkTestBase.java | 35 ++--- .../Elasticsearch1ApiCallBridge.java | 10 +- .../elasticsearch/ElasticsearchSink.java | 3 +- .../ElasticsearchSinkITCase.java | 30 +++-- .../Elasticsearch2ApiCallBridge.java | 9 +- .../elasticsearch2/ElasticsearchSink.java | 2 +- .../ElasticsearchSinkITCase.java | 33 +++-- .../Elasticsearch5ApiCallBridge.java | 9 +- .../elasticsearch5/ElasticsearchSink.java | 2 +- .../ElasticsearchSinkITCase.java | 37 +++--- .../Elasticsearch6ApiCallBridge.java | 9 +- .../elasticsearch6/ElasticsearchSink.java | 13 +- ...eddedElasticsearchNodeEnvironmentImpl.java | 5 +- .../ElasticsearchSinkITCase.java | 123 ++++-------------- .../examples/ElasticsearchSinkExample.java | 81 ------------ .../src/test/resources/log4j-test.properties | 3 - .../test-scripts/elasticsearch-common.sh | 20 +-- 21 files changed, 161 insertions(+), 301 deletions(-) delete mode 100644 flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 20a7d7138ee51..f327571f1a1af 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -243,7 +243,7 @@ input.addSink(new ElasticsearchSink(httpHosts, new ElasticsearchSinkFunction[Str -Note how `TransportClient` based version use a `Map` of `String`s is used to configure the `ElasticsearchSink`. +Note how `TransportClient` based versions use a `Map` of `String`s to configure the `ElasticsearchSink`. 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 diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 90d84f3635a10..650f4c640e30f 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -36,9 +36,11 @@ *

Implementations are allowed to be stateful. For example, for Elasticsearch 1.x, since connecting via an embedded node * is allowed, the call bridge will hold reference to the created embedded node. Each instance of the sink will hold * exactly one instance of the call bridge, and state cleanup is performed when the sink is closed. + * + * @param The Elasticsearch client, that implements {@link AutoCloseable}. */ @Internal -public abstract class ElasticsearchApiCallBridge implements Serializable { +public interface ElasticsearchApiCallBridge extends Serializable { /** * Creates an Elasticsearch client implementing {@link AutoCloseable}. @@ -46,9 +48,16 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - public abstract AutoCloseable createClient(Map clientConfig); + C createClient(Map clientConfig); - public abstract BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener); + /** + * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. + * + * @param client the Elasticsearch client. + * @param listener the bulk processor listender. + * @return the bulk processor builder. + */ + BulkProcessor.Builder createBulkProcessorBuilder(C client, BulkProcessor.Listener listener); /** * Extracts the cause of failure of a bulk item action. @@ -56,7 +65,7 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param bulkItemResponse the bulk item response to extract cause of failure * @return the extracted {@link Throwable} from the response ({@code null} is the response is successful). */ - public abstract @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); + @Nullable Throwable extractFailureCauseFromBulkItemResponse(BulkItemResponse bulkItemResponse); /** * Set backoff-related configurations on the provided {@link BulkProcessor.Builder}. @@ -65,14 +74,14 @@ public abstract class ElasticsearchApiCallBridge implements Serializable { * @param builder the {@link BulkProcessor.Builder} to configure. * @param flushBackoffPolicy user-provided backoff retry settings ({@code null} if the user disabled backoff retries). */ - public abstract void configureBulkProcessorBackoff( + void configureBulkProcessorBackoff( BulkProcessor.Builder builder, @Nullable ElasticsearchSinkBase.BulkFlushBackoffPolicy flushBackoffPolicy); /** * Perform any necessary state cleanup. */ - public void cleanup() { + default void cleanup() { // nothing to cleanup by default } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 98304844410b0..b96b79e1abec2 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -61,9 +61,10 @@ * for example, to create a Elasticsearch {@link Client} or {@RestHighLevelClient}, handle failed item responses, etc. * * @param Type of the elements handled by this sink + * @param Type of the Elasticsearch client, which implements {@link AutoCloseable} */ @Internal -public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { +public abstract class ElasticsearchSinkBase extends RichSinkFunction implements CheckpointedFunction { private static final long serialVersionUID = -1007596293618451942L; @@ -161,7 +162,7 @@ public void setDelayMillis(long delayMillis) { // ------------------------------------------------------------------------ /** Call bridge for different version-specific. */ - private final ElasticsearchApiCallBridge callBridge; + private final ElasticsearchApiCallBridge callBridge; /** * Number of pending action requests not yet acknowledged by Elasticsearch. @@ -175,7 +176,7 @@ public void setDelayMillis(long delayMillis) { private AtomicLong numPendingRequests = new AtomicLong(0); /** Elasticsearch client created using the call bridge. */ - private transient AutoCloseable client; + private transient C client; /** Bulk processor to buffer and send requests to Elasticsearch, created using the client. */ private transient BulkProcessor bulkProcessor; diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java index 460e939fab9db..369d26a735a64 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBaseTest.java @@ -411,7 +411,7 @@ public void testDoesNotWaitForPendingRequestsIfFlushingDisabled() throws Excepti testHarness.close(); } - private static class DummyElasticsearchSink extends ElasticsearchSinkBase { + private static class DummyElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 5051907841570096991L; @@ -531,17 +531,17 @@ public Object answer(InvocationOnMock invocationOnMock) throws Throwable { } } - private static class DummyElasticsearchApiCallBridge extends ElasticsearchApiCallBridge { + private static class DummyElasticsearchApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -4272760730959041699L; @Override - public AutoCloseable createClient(Map clientConfig) { + public Client createClient(Map clientConfig) { return mock(Client.class); } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { + public BulkProcessor.Builder createBulkProcessorBuilder(Client client, BulkProcessor.Listener listener) { return null; } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index df3779b1fd49c..2259d34de5b92 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -34,7 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.InetSocketAddress; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -45,8 +44,11 @@ /** * Environment preparation and suite of tests for version-specific {@link ElasticsearchSinkBase} implementations. + * + * @param Elasticsearch client type + * @param The address type to use */ -public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { +public abstract class ElasticsearchSinkTestBase extends AbstractTestBase { private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchSinkTestBase.class); @@ -85,9 +87,9 @@ public static void shutdown() throws Exception { } /** - * Tests that the Elasticsearch sink works properly using a {@link TransportClient}. + * Tests that the Elasticsearch sink works properly. */ - public void runTransportClientTest() throws Exception { + public void runElasticsearchSinkTest() throws Exception { final String index = "transport-client-test-index"; final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -112,9 +114,9 @@ public void runTransportClientTest() throws Exception { } /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is {@code null}. */ - public void runNullTransportClientTest() throws Exception { + public void runNullAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); userConfig.put("cluster.name", "my-transport-client-cluster"); @@ -130,9 +132,9 @@ public void runNullTransportClientTest() throws Exception { } /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. + * Tests that the Elasticsearch sink fails eagerly if the provided list of addresses is empty. */ - public void runEmptyTransportClientTest() throws Exception { + public void runEmptyAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); userConfig.put("cluster.name", "my-transport-client-cluster"); @@ -140,7 +142,7 @@ public void runEmptyTransportClientTest() throws Exception { try { createElasticsearchSink( userConfig, - Collections.emptyList(), + Collections.emptyList(), new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); } catch (IllegalArgumentException expectedException) { // test passes @@ -153,7 +155,7 @@ public void runEmptyTransportClientTest() throws Exception { /** * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. */ - public void runTransportClientFailsTest() throws Exception { + public void runInvalidElasticsearchClusterTest() throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); @@ -168,7 +170,7 @@ public void runTransportClientFailsTest() throws Exception { try { env.execute("Elasticsearch Transport Client Test"); } catch (JobExecutionException expectedException) { - assertTrue(expectedException.getCause().getMessage().contains("not connected to any Elasticsearch nodes")); + expectedException.printStackTrace(); return; } @@ -176,9 +178,10 @@ public void runTransportClientFailsTest() throws Exception { } /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ - protected abstract ElasticsearchSinkBase createElasticsearchSink(Map userConfig, - List transportAddresses, - ElasticsearchSinkFunction elasticsearchSinkFunction); + protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( + Map userConfig, + List addresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction); /** * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch node. @@ -186,6 +189,6 @@ protected abstract ElasticsearchSinkBase createElasticsearchSink(MapThis case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(Map, List, ElasticsearchSinkFunction)} * because the Elasticsearch Java API to do so is incompatible across different versions. */ - protected abstract ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception; + protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( + Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; } 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 index 28d5f345ff007..4f1cd086d8fa2 100644 --- 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 @@ -42,7 +42,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 1.x. */ @Internal -public class Elasticsearch1ApiCallBridge extends ElasticsearchApiCallBridge { +public class Elasticsearch1ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -2632363720584123682L; @@ -70,7 +70,7 @@ public class Elasticsearch1ApiCallBridge extends ElasticsearchApiCallBridge { } @Override - public AutoCloseable createClient(Map clientConfig) { + public Client createClient(Map clientConfig) { if (transportAddresses == null) { // Make sure that we disable http access to our embedded node @@ -85,7 +85,7 @@ public AutoCloseable createClient(Map clientConfig) { .data(false) .node(); - AutoCloseable client = node.client(); + Client client = node.client(); if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch client from embedded node"); @@ -116,8 +116,8 @@ public AutoCloseable createClient(Map clientConfig) { } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - return BulkProcessor.builder((Client) client, listener); + public BulkProcessor.Builder createBulkProcessorBuilder(Client client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client, listener); } @Override 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 e8eccd978f490..d5e1d1fdc1219 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 @@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.node.Node; @@ -64,7 +65,7 @@ * @param Type of the elements handled by this sink */ @PublicEvolving -public class ElasticsearchSink extends ElasticsearchSinkBase { +public class ElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 1L; 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 54892909abca3..c3e9b01bfac96 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 @@ -42,26 +42,26 @@ /** * IT Cases for the {@link ElasticsearchSink}. */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { @Test - public void testTransportClient() throws Exception { - runTransportClientTest(); + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); } @Test - public void testNullTransportClient() throws Exception { - runNullTransportClientTest(); + public void testNullAddresses() throws Exception { + runNullAddressesTest(); } @Test - public void testEmptyTransportClient() throws Exception { - runEmptyTransportClientTest(); + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); } @Test - public void testTransportClientFails() throws Exception{ - runTransportClientFailsTest(); + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); } // -- Tests specific to Elasticsearch 1.x -- @@ -102,15 +102,17 @@ public void testDeprecatedIndexRequestBuilderVariant() throws Exception { } @Override - protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, - List transportAddresses, - ElasticsearchSinkFunction elasticsearchSinkFunction) { + protected ElasticsearchSinkBase, Client> createElasticsearchSink( + Map userConfig, + List transportAddresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { return new ElasticsearchSink<>(userConfig, ElasticsearchUtils.convertInetSocketAddresses(transportAddresses), elasticsearchSinkFunction); } @Override - protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + protected ElasticsearchSinkBase, Client> createElasticsearchSinkForEmbeddedNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { // Elasticsearch 1.x requires this setting when using // LocalTransportAddress to connect to a local embedded node 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 index 80c1b3acf2204..73a69ebde3470 100644 --- 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 @@ -26,7 +26,6 @@ import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; @@ -44,7 +43,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 2.x. */ @Internal -public class Elasticsearch2ApiCallBridge extends ElasticsearchApiCallBridge { +public class Elasticsearch2ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = 2638252694744361079L; @@ -63,7 +62,7 @@ public class Elasticsearch2ApiCallBridge extends ElasticsearchApiCallBridge { } @Override - public AutoCloseable createClient(Map clientConfig) { + public TransportClient createClient(Map clientConfig) { Settings settings = Settings.settingsBuilder().put(clientConfig).build(); TransportClient transportClient = TransportClient.builder().settings(settings).build(); @@ -84,8 +83,8 @@ public AutoCloseable createClient(Map clientConfig) { } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - return BulkProcessor.builder((Client) client, listener); + public BulkProcessor.Builder createBulkProcessorBuilder(TransportClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client, listener); } @Override 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 ffccacf40ace1..a911905ac0a1c 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 @@ -58,7 +58,7 @@ * @param Type of the elements handled by this sink */ @PublicEvolving -public class ElasticsearchSink extends ElasticsearchSinkBase { +public class ElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 1L; 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 7ded893be3a03..9d996ad212e9a 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 @@ -17,10 +17,12 @@ package org.apache.flink.streaming.connectors.elasticsearch2; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.elasticsearch.client.transport.TransportClient; import org.junit.Test; import java.net.InetAddress; @@ -32,38 +34,41 @@ /** * IT cases for the {@link ElasticsearchSink}. */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { @Test - public void testTransportClient() throws Exception { - runTransportClientTest(); + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); } @Test - public void testNullTransportClient() throws Exception { - runNullTransportClientTest(); + public void testNullAddresses() throws Exception { + runNullAddressesTest(); } @Test - public void testEmptyTransportClient() throws Exception { - runEmptyTransportClientTest(); + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); } @Test - public void testTransportClientFails() throws Exception{ - runTransportClientFailsTest(); + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); } @Override - protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, - List transportAddresses, - ElasticsearchSinkFunction elasticsearchSinkFunction) { + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSink( + Map userConfig, + List transportAddresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + return new ElasticsearchSink<>(userConfig, transportAddresses, elasticsearchSinkFunction); } @Override - protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForEmbeddedNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { List transports = new ArrayList<>(); transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); 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 index 1e73feb9e4377..a3453ec44452a 100644 --- 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 @@ -26,7 +26,6 @@ import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; -import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; @@ -47,7 +46,7 @@ * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 5.x. */ @Internal -public class Elasticsearch5ApiCallBridge extends ElasticsearchApiCallBridge { +public class Elasticsearch5ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -5222683870097809633L; @@ -66,7 +65,7 @@ public class Elasticsearch5ApiCallBridge extends ElasticsearchApiCallBridge { } @Override - public AutoCloseable createClient(Map clientConfig) { + public TransportClient 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) @@ -90,8 +89,8 @@ public AutoCloseable createClient(Map clientConfig) { } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - return BulkProcessor.builder((Client) client, listener); + public BulkProcessor.Builder createBulkProcessorBuilder(TransportClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client, listener); } @Override 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 6c09337227a9d..b99b353925509 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 @@ -59,7 +59,7 @@ * @param Type of the elements handled by this sink */ @PublicEvolving -public class ElasticsearchSink extends ElasticsearchSinkBase { +public class ElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 1L; 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 ad7c664cac748..f7ed056706b6c 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 @@ -18,10 +18,12 @@ package org.apache.flink.streaming.connectors.elasticsearch5; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; +import org.elasticsearch.client.transport.TransportClient; import org.junit.Test; import java.net.InetAddress; @@ -33,43 +35,44 @@ /** * IT cases for the {@link ElasticsearchSink}. */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { @Test - public void testTransportClient() throws Exception { - runTransportClientTest(); + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); } @Test - public void testNullTransportClient() throws Exception { - runNullTransportClientTest(); + public void testNullAddresses() throws Exception { + runNullAddressesTest(); } @Test - public void testEmptyTransportClient() throws Exception { - runEmptyTransportClientTest(); + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); } @Test - public void testTransportClientFails() throws Exception { - runTransportClientFailsTest(); + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); } @Override - protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, - List transportAddresses, - ElasticsearchSinkFunction elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, transportAddresses, elasticsearchSinkFunction); + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSink( + Map userConfig, + List addresses, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return new ElasticsearchSink<>(userConfig, addresses, elasticsearchSinkFunction); } @Override - protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { - + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForEmbeddedNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { List transports = new ArrayList<>(); transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction); } - } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 2cb4ea03a13f8..72069517347d5 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -39,7 +39,7 @@ /** * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ -public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { +public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -5222683870097809633L; @@ -56,7 +56,7 @@ public class Elasticsearch6ApiCallBridge extends ElasticsearchApiCallBridge { } @Override - public AutoCloseable createClient(Map clientConfig) { + public RestHighLevelClient createClient(Map clientConfig) { RestHighLevelClient rhlClient = new RestHighLevelClient(RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]))); @@ -68,9 +68,8 @@ public AutoCloseable createClient(Map clientConfig) { } @Override - public BulkProcessor.Builder createBulkProcessorBuilder(AutoCloseable client, BulkProcessor.Listener listener) { - RestHighLevelClient rhlClient = (RestHighLevelClient) client; - return BulkProcessor.builder(rhlClient::bulkAsync, listener); + public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { + return BulkProcessor.builder(client::bulkAsync, listener); } @Override diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 3f75b5ff190d7..38d8f47819032 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.elasticsearch6; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.streaming.connectors.elasticsearch.ActionRequestFailureHandler; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; @@ -24,7 +25,6 @@ import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; -import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.RestHighLevelClient; @@ -38,10 +38,6 @@ *

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

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

Internally, the sink will use a {@link BulkProcessor} to send {@link ActionRequest ActionRequests}. * This will buffer elements before sending a request to the cluster. The behaviour of the * {@code BulkProcessor} can be configured using these config keys: @@ -58,7 +54,8 @@ * * @param Type of the elements handled by this sink */ -public class ElasticsearchSink extends ElasticsearchSinkBase { +@PublicEvolving +public class ElasticsearchSink extends ElasticsearchSinkBase { private static final long serialVersionUID = 1L; @@ -66,7 +63,7 @@ public class ElasticsearchSink extends ElasticsearchSinkBase { * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. * * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element - * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. */ public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { @@ -78,7 +75,7 @@ public ElasticsearchSink(Map userConfig, List httpHost * * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element * @param failureHandler This is used to handle failed {@link ActionRequest} - * @param httpHosts The list of {@HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. */ public ElasticsearchSink( Map userConfig, diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java index f419b41b49663..8dc6216804949 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/EmbeddedElasticsearchNodeEnvironmentImpl.java @@ -21,7 +21,6 @@ import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSinkITCase; import org.elasticsearch.client.Client; -import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.node.InternalSettingsPreparer; import org.elasticsearch.node.Node; @@ -44,11 +43,9 @@ public void start(File tmpDataFolder, String clusterName) throws Exception { if (node == null) { Settings settings = Settings.builder() .put("cluster.name", clusterName) - .put("http.enabled", false) + .put("http.enabled", true) .put("path.home", tmpDataFolder.getParent()) .put("path.data", tmpDataFolder.getAbsolutePath()) - .put(NetworkModule.HTTP_TYPE_KEY, Netty4Plugin.NETTY_HTTP_TRANSPORT_NAME) - .put(NetworkModule.TRANSPORT_TYPE_KEY, Netty4Plugin.NETTY_TRANSPORT_NAME) .build(); node = new PluginNode(settings); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 217077171c63d..64a4f486f083c 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -19,134 +19,59 @@ package org.apache.flink.streaming.connectors.elasticsearch6; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.runtime.client.JobExecutionException; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkTestBase; -import org.apache.flink.streaming.connectors.elasticsearch.testutils.SourceSinkDataTestKit; import org.apache.http.HttpHost; -import org.elasticsearch.client.Client; import org.elasticsearch.client.RestHighLevelClient; +import org.junit.Test; -import java.net.InetSocketAddress; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - /** * IT cases for the {@link ElasticsearchSink}. */ -public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - - /** - * Tests that the Elasticsearch sink works properly using a {@link RestHighLevelClient}. - */ - public void runTransportClientTest() throws Exception { - final String index = "transport-client-test-index"; - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - - source.addSink(createElasticsearchSinkForEmbeddedNode(userConfig, - new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); +public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { - env.execute("Elasticsearch RestHighLevelClient Test"); - - // verify the results - Client client = embeddedNodeEnv.getClient(); - SourceSinkDataTestKit.verifyProducedSinkData(client, index); - - client.close(); + @Test + public void testElasticsearchSink() throws Exception { + runElasticsearchSinkTest(); } - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is {@code null}. - */ - public void runNullTransportClientTest() throws Exception { - try { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - createElasticsearchSink6(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { - // test passes - return; - } - - fail(); + @Test + public void testNullAddresses() throws Exception { + runNullAddressesTest(); } - /** - * Tests that the Elasticsearch sink fails eagerly if the provided list of transport addresses is empty. - */ - public void runEmptyTransportClientTest() throws Exception { - try { - Map userConfig = new HashMap<>(); - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - createElasticsearchSink6(userConfig, - Collections.emptyList(), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { - // test passes - return; - } - - fail(); + @Test + public void testEmptyAddresses() throws Exception { + runEmptyAddressesTest(); } - /** - * Tests whether the Elasticsearch sink fails when there is no cluster to connect to. - */ - public void runTransportClientFailsTest() throws Exception { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - - 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(); + @Test + public void testInvalidElasticsearchCluster() throws Exception{ + runInvalidElasticsearchClusterTest(); } @Override - protected ElasticsearchSinkBase createElasticsearchSink(Map userConfig, List transportAddresses, ElasticsearchSinkFunction elasticsearchSinkFunction) { - return null; + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction> elasticsearchSinkFunction) { + + return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); } @Override - protected ElasticsearchSinkBase createElasticsearchSinkForEmbeddedNode(Map userConfig, ElasticsearchSinkFunction elasticsearchSinkFunction) throws Exception { + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + ArrayList httpHosts = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); } - - private ElasticsearchSinkBase createElasticsearchSink6( - Map userConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); - } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java deleted file mode 100644 index de1670feac4f5..0000000000000 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/examples/ElasticsearchSinkExample.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.streaming.connectors.elasticsearch6.examples; - -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; -import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; -import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; - -import org.apache.http.HttpHost; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.client.Requests; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This example shows how to use the Elasticsearch Sink. Before running it you must ensure that - * you have a cluster named "elasticsearch" running or change the name of cluster in the config map. - */ -public class ElasticsearchSinkExample { - - public static void main(String[] args) throws Exception { - - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - - DataStream source = env.generateSequence(0, 20).map(new MapFunction() { - @Override - public String map(Long value) throws Exception { - return "message #" + value; - } - }); - - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - - List httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); - - source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element)); - } - })); - - env.execute("Elasticsearch Sink Example"); - } - - private static IndexRequest createIndexRequest(String element) { - Map json = new HashMap<>(); - json.put("data", element); - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .id(element) - .source(json); - } -} diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties index 20551848eea24..fcd8654666847 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/resources/log4j-test.properties @@ -22,6 +22,3 @@ log4j.appender.testlogger=org.apache.log4j.ConsoleAppender log4j.appender.testlogger.target=System.err log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n - -# suppress the irrelevant (wrong) warnings from the netty channel handler -log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, testlogger diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 7b627fe9de407..900b73664c26a 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -42,15 +42,19 @@ function setup_elasticsearch { } function verify_elasticsearch_process_exist { - local elasticsearchProcess=$(jps | grep Elasticsearch | awk '{print $2}') + while : ; do + local elasticsearchProcess=$(jps | grep Elasticsearch | awk '{print $2}') - # make sure the elasticsearch node is actually running - if [ "$elasticsearchProcess" != "Elasticsearch" ]; then - echo "Elasticsearch node is not running." - exit 1 - else - echo "Elasticsearch node is running." - fi + echo "Waiting for Elasticsearch node to start ..." + + # make sure the elasticsearch node is actually running + if [ "$elasticsearchProcess" != "Elasticsearch" ]; then + sleep 1 + else + echo "Elasticsearch node is running." + break + fi + done } function verify_result { From 246143258a663aff49bf00539659a7e724b8b9f3 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 15:30:05 +0800 Subject: [PATCH 05/21] fixup! Fix errorneous Javadocs --- .../elasticsearch/ElasticsearchSinkBase.java | 11 +++++++++-- .../connectors/elasticsearch6/ElasticsearchSink.java | 2 ++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index b96b79e1abec2..ae5c45ac9cf0e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -32,6 +32,7 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.client.Client; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -58,7 +59,7 @@ * *

The version specific API calls for different Elasticsearch versions should be defined by a concrete implementation of * a {@link ElasticsearchApiCallBridge}, which is provided to the constructor of this class. This call bridge is used, - * for example, to create a Elasticsearch {@link Client} or {@RestHighLevelClient}, handle failed item responses, etc. + * for example, to create a Elasticsearch {@link Client}, handle failed item responses, etc. * * @param Type of the elements handled by this sink * @param Type of the Elasticsearch client, which implements {@link AutoCloseable} @@ -142,7 +143,13 @@ public void setDelayMillis(long delayMillis) { // User-facing API and configuration // ------------------------------------------------------------------------ - /** The user specified config map that we forward to Elasticsearch when we create the {@link Client}. */ + /** + * The config map that contains configuration for the bulk flushing behaviours. + * + * For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this config + * map would also contain Elasticsearch-shipped configuration, and therefore this config map + * would also be forwarded when creating the Elasticsearch client. + */ private final Map userConfig; /** The function that is used to construct multiple {@link ActionRequest ActionRequests} from each incoming element. */ diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 38d8f47819032..7c985aa190f42 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -62,6 +62,7 @@ public class ElasticsearchSink extends ElasticsearchSinkBase userConfig, List httpHost /** * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. * + * @param userConfig user configuration to configure bulk flushing behaviour. * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element * @param failureHandler This is used to handle failed {@link ActionRequest} * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. From 5036221e0c3353e7038ca5fe775d7b8851f9cb60 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 15:32:52 +0800 Subject: [PATCH 06/21] fixup! Allow overriding rest client builder for custom configuration --- .../elasticsearch6/Elasticsearch6ApiCallBridge.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 72069517347d5..668bed7e6bae6 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.common.unit.TimeValue; import org.slf4j.Logger; @@ -57,8 +58,7 @@ public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge clientConfig) { - RestHighLevelClient rhlClient = - new RestHighLevelClient(RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()]))); + RestHighLevelClient rhlClient = new RestHighLevelClient(createRestClientBuilder()); if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); @@ -67,6 +67,15 @@ public RestHighLevelClient createClient(Map clientConfig) { return rhlClient; } + /** + * Users can override this method to have custom configuration for the rest client. + * + * @return the builder for a {@link RestHighLevelClient}. + */ + protected RestClientBuilder createRestClientBuilder() { + return RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + } + @Override public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { return BulkProcessor.builder(client::bulkAsync, listener); From 682d383332fce527f245e3ff1effeaed14716197 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 15:34:13 +0800 Subject: [PATCH 07/21] fixup! Remove non-used imports --- .../ElasticsearchSinkTestBase.java | 2 - .../Elasticsearch6ApiCallBridge.java | 24 ++++++----- .../elasticsearch6/ElasticsearchSink.java | 43 ++++++++++++++++--- .../elasticsearch6/RestClientFactory.java | 39 +++++++++++++++++ 4 files changed, 90 insertions(+), 18 deletions(-) create mode 100644 flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 2259d34de5b92..fd8a54c053ac8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -26,7 +26,6 @@ import org.apache.flink.util.InstantiationUtil; import org.elasticsearch.client.Client; -import org.elasticsearch.client.transport.TransportClient; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.ClassRule; @@ -39,7 +38,6 @@ import java.util.List; import java.util.Map; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; /** diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 668bed7e6bae6..1912134171b5f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.connectors.elasticsearch6; +import org.apache.flink.annotation.Internal; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchApiCallBridge; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.util.Preconditions; @@ -40,6 +41,7 @@ /** * Implementation of {@link ElasticsearchApiCallBridge} for Elasticsearch 6 and later versions. */ +@Internal public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge { private static final long serialVersionUID = -5222683870097809633L; @@ -51,14 +53,23 @@ public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge httpHosts; - Elasticsearch6ApiCallBridge(List httpHosts) { + /** + * The factory to configure the rest client. + */ + private final RestClientFactory restClientFactory; + + Elasticsearch6ApiCallBridge(List httpHosts, RestClientFactory restClientFactory) { Preconditions.checkArgument(httpHosts != null && !httpHosts.isEmpty()); this.httpHosts = httpHosts; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); } @Override public RestHighLevelClient createClient(Map clientConfig) { - RestHighLevelClient rhlClient = new RestHighLevelClient(createRestClientBuilder()); + RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); + restClientFactory.configureRestClientBuilder(builder); + + RestHighLevelClient rhlClient = new RestHighLevelClient(builder); if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); @@ -67,15 +78,6 @@ public RestHighLevelClient createClient(Map clientConfig) { return rhlClient; } - /** - * Users can override this method to have custom configuration for the rest client. - * - * @return the builder for a {@link RestHighLevelClient}. - */ - protected RestClientBuilder createRestClientBuilder() { - return RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); - } - @Override public BulkProcessor.Builder createBulkProcessorBuilder(RestHighLevelClient client, BulkProcessor.Listener listener) { return BulkProcessor.builder(client::bulkAsync, listener); diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 7c985aa190f42..4bdfc521c04fd 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -62,29 +62,62 @@ public class ElasticsearchSink extends ElasticsearchSinkBaseThis constructor uses the default configurations of the {@link RestHighLevelClient}. For custom + * configuration, please use the {@link #ElasticsearchSink(Map, List, ElasticsearchSinkFunction, RestClientFactory)} + * constructor. + * * @param userConfig user configuration to configure bulk flushing behaviour. * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. */ public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { - this(userConfig, httpHosts, elasticsearchSinkFunction, new NoOpFailureHandler()); + this( + userConfig, + httpHosts, + elasticsearchSinkFunction, + restClientBuilder -> {} // a no-op rest client factory, that uses the default configurations + ); + } + + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param userConfig user configuration to configure bulk flushing behaviour. + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + * @param restClientFactory the factory that configures the rest client. + */ + public ElasticsearchSink( + Map userConfig, + List httpHosts, + ElasticsearchSinkFunction elasticsearchSinkFunction, + RestClientFactory restClientFactory) { + + this( + userConfig, + httpHosts, + elasticsearchSinkFunction, + new NoOpFailureHandler(), + restClientFactory); } /** * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. * * @param userConfig user configuration to configure bulk flushing behaviour. - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element - * @param failureHandler This is used to handle failed {@link ActionRequest} * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + * @param failureHandler This is used to handle failed {@link ActionRequest}. + * @param restClientFactory the factory that configures the rest client. */ public ElasticsearchSink( Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler) { + ActionRequestFailureHandler failureHandler, + RestClientFactory restClientFactory) { - super(new Elasticsearch6ApiCallBridge(httpHosts), userConfig, elasticsearchSinkFunction, failureHandler); + super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), userConfig, elasticsearchSinkFunction, failureHandler); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java new file mode 100644 index 0000000000000..82673cd644a54 --- /dev/null +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.connectors.elasticsearch6; + +import org.apache.flink.annotation.PublicEvolving; +import org.elasticsearch.client.RestClientBuilder; + +import java.io.Serializable; + +/** + * A factory that is used to configure the {@link org.elasticsearch.client.RestHighLevelClient} internally + * used in the {@link ElasticsearchSink}. + */ +@PublicEvolving +public interface RestClientFactory extends Serializable { + + /** + * Configures the rest client builder. + * + * @param restClientBuilder the configured rest client builder. + */ + void configureRestClientBuilder(RestClientBuilder restClientBuilder); + +} From 1982b43ff0cb083a167de2e5a9ead30da7ade49a Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 16:39:49 +0800 Subject: [PATCH 08/21] fixup! Use Builder pattern for instantiating Elasticsearch 6.x sinks --- .../elasticsearch6/ElasticsearchSink.java | 83 +++++++++---------- .../ElasticsearchSinkITCase.java | 11 ++- .../tests/Elasticsearch6SinkExample.java | 17 ++-- 3 files changed, 54 insertions(+), 57 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 4bdfc521c04fd..c60ac4d922062 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -23,11 +23,13 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; +import org.apache.flink.util.Preconditions; import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.client.RestHighLevelClient; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -62,62 +64,51 @@ public class ElasticsearchSink extends ElasticsearchSinkBaseThis constructor uses the default configurations of the {@link RestHighLevelClient}. For custom - * configuration, please use the {@link #ElasticsearchSink(Map, List, ElasticsearchSinkFunction, RestClientFactory)} - * constructor. - * - * @param userConfig user configuration to configure bulk flushing behaviour. - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element - * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. - */ - public ElasticsearchSink(Map userConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { - - this( - userConfig, - httpHosts, - elasticsearchSinkFunction, - restClientBuilder -> {} // a no-op rest client factory, that uses the default configurations - ); - } - - /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. - * - * @param userConfig user configuration to configure bulk flushing behaviour. + * @param bulkRequestsConfig user configuration to configure bulk flushing behaviour. * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + * @param failureHandler This is used to handle failed {@link ActionRequest}. * @param restClientFactory the factory that configures the rest client. */ - public ElasticsearchSink( - Map userConfig, + private ElasticsearchSink( + Map bulkRequestsConfig, List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction, + ActionRequestFailureHandler failureHandler, RestClientFactory restClientFactory) { - this( - userConfig, - httpHosts, - elasticsearchSinkFunction, - new NoOpFailureHandler(), - restClientFactory); + super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); } - /** - * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. - * - * @param userConfig user configuration to configure bulk flushing behaviour. - * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. - * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. - * @param failureHandler This is used to handle failed {@link ActionRequest}. - * @param restClientFactory the factory that configures the rest client. - */ - public ElasticsearchSink( - Map userConfig, - List httpHosts, - ElasticsearchSinkFunction elasticsearchSinkFunction, - ActionRequestFailureHandler failureHandler, - RestClientFactory restClientFactory) { + @PublicEvolving + public static class Builder { + + private final List httpHosts; + private final ElasticsearchSinkFunction elasticsearchSinkFunction; + + private Map bulkRequestsConfig = new HashMap<>(); + private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); + private RestClientFactory restClientFactory = restClientBuilder -> {}; + + public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { + this.httpHosts = Preconditions.checkNotNull(httpHosts); + this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); + } + + public void setBulkRequestsConfig(Map bulkRequestsConfig) { + this.bulkRequestsConfig = bulkRequestsConfig; + } + + public void setFailureHandler(ActionRequestFailureHandler failureHandler) { + this.failureHandler = failureHandler; + } + + public void setRestClientFactory(RestClientFactory restClientFactory) { + this.restClientFactory = restClientFactory; + } - super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), userConfig, elasticsearchSinkFunction, failureHandler); + public ElasticsearchSink build() { + return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); + } } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 64a4f486f083c..8c456143d6a8b 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -62,7 +62,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr List httpHosts, ElasticsearchSinkFunction> elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkRequestsConfig(userConfig); + + return builder.build(); } @Override @@ -72,6 +75,10 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr ArrayList httpHosts = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); - return new ElasticsearchSink<>(userConfig, httpHosts, elasticsearchSinkFunction); + + ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); + builder.setBulkRequestsConfig(userConfig); + + return builder.build(); } } diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java index 5544ea58f3e42..8e0c59f865504 100644 --- a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java @@ -22,7 +22,6 @@ import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; @@ -62,19 +61,19 @@ public String map(Long value) throws Exception { } }); - Map userConfig = new HashMap<>(); + Map bulkRequestsConfig = new HashMap<>(); // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); + bulkRequestsConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); List httpHosts = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); - source.addSink(new ElasticsearchSink<>(userConfig, httpHosts, new ElasticsearchSinkFunction() { - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element, parameterTool)); - } - })); + ElasticsearchSink.Builder esSinkBuilder = new ElasticsearchSink.Builder<>( + httpHosts, + (String element, RuntimeContext ctx, RequestIndexer indexer) -> indexer.add(createIndexRequest(element, parameterTool))); + esSinkBuilder.setBulkRequestsConfig(bulkRequestsConfig); + + source.addSink(esSinkBuilder.build()); env.execute("Elasticsearch 6.x end to end sink test example"); } From a27dce5cefb7e02e6cdf3f674b2c89d3d04c9b07 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 17:01:17 +0800 Subject: [PATCH 09/21] fixup! Fix checkstyle --- .../connectors/elasticsearch/ElasticsearchSinkBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index ae5c45ac9cf0e..1c5b1f3d60953 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -146,7 +146,7 @@ public void setDelayMillis(long delayMillis) { /** * The config map that contains configuration for the bulk flushing behaviours. * - * For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this config + *

For {@link org.elasticsearch.client.transport.TransportClient} based implementations, this config * map would also contain Elasticsearch-shipped configuration, and therefore this config map * would also be forwarded when creating the Elasticsearch client. */ From afd394a5e9370b917aa5ce3a70b7189624da53b2 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 17:01:36 +0800 Subject: [PATCH 10/21] fixup! Improve documentation --- docs/dev/connectors/elasticsearch.md | 108 ++++++++++++++++++++------- 1 file changed, 82 insertions(+), 26 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index f327571f1a1af..eb4f74f6b805e 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -151,22 +151,47 @@ List httpHost = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")); -input.addSink(new ElasticsearchSink<>(httpHosts, new ElasticsearchSinkFunction() { - public IndexRequest createIndexRequest(String element) { - Map json = new HashMap<>(); - json.put("data", element); - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .source(json); - } - - @Override - public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { - indexer.add(createIndexRequest(element)); +Map bulkRequestsConfig = new HashMap<>(); +// This instructs the sink to emit after every element, otherwise they would be buffered +bulkRequestsConfig.put("bulk.flush.max.actions", "1"); + +// use a ElasticsearchSink.Builder to create an ElasticsearchSink +ElasticsearchSink.Builder esSinkBuilder = new ElasticsearchSink.Builder<>( + httpHosts, + new ElasticsearchSinkFunction() { + public IndexRequest createIndexRequest(String element) { + Map json = new HashMap<>(); + json.put("data", element); + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json); + } + + @Override + public void process(String element, RuntimeContext ctx, RequestIndexer indexer) { + indexer.add(createIndexRequest(element)); + } } -}));{% endhighlight %} +); + +// configuration for the bulk requests +builder.setBulkRequestsConfig(bulkRequestsConfig); + +// provide a RestClientFactory for custom configuration on the internally created REST client +builder.setRestClientBuilder( + restClientBuilder -> { + restClientBuilder.setDefaultHeaders(...) + restClientBuilder.setMaxRetryTimeoutMillis(...) + restClientBuilder.setPathPrefix(...) + restClientBuilder.setHttpClientConfigCallback(...) + } +); + +// finally, build and add the sink to the job's pipeline +input.addSink(esSinkBuilder.build()); +{% endhighlight %}

{% highlight scala %} @@ -228,27 +253,58 @@ val httpHosts = new java.util.ArrayList[HttpHost] httpHosts.add(new HttpHost("127.0.0.1", 9300, "http")) httpHosts.add(new HttpHost("10.2.3.1", 9300, "http")) -input.addSink(new ElasticsearchSink(httpHosts, new ElasticsearchSinkFunction[String] { - def createIndexRequest(element: String): IndexRequest = { - val json = new java.util.HashMap[String, String] - json.put("data", element) - - return Requests.indexRequest() - .index("my-index") - .type("my-type") - .source(json) +val bulkRequestsConfig = new java.util.HashMap[String, String] +// This instructs the sink to emit after every element, otherwise they would be buffered +bulkRequestsConfig.put("bulk.flush.max.actions", "1") + +val esSinkBuilder = new ElasticsearchSink.Builer[String]( + httpHosts, + new ElasticsearchSinkFunction[String] { + def createIndexRequest(element: String): IndexRequest = { + val json = new java.util.HashMap[String, String] + json.put("data", element) + + return Requests.indexRequest() + .index("my-index") + .type("my-type") + .source(json) + } } -})) +) + +// configuration for the bulk requests +builder.setBulkRequestsConfig(bulkRequestsConfig) + +// provide a RestClientFactory for custom configuration on the internally created REST client +builder.setRestClientBuilder( + restClientBuilder -> { + restClientBuilder.setDefaultHeaders(...) + restClientBuilder.setMaxRetryTimeoutMillis(...) + restClientBuilder.setPathPrefix(...) + restClientBuilder.setHttpClientConfigCallback(...) + } +) + +// finally, build and add the sink to the job's pipeline +input.addSink(esSinkBuilder.build) {% endhighlight %}
-Note how `TransportClient` based versions use a `Map` of `String`s to configure the `ElasticsearchSink`. +For Elasticsearch versions that still uses the now deprecated `TransportClient` to communicate +with the Elasticsearch cluster (i.e., versions equal or below 5.x), note how a `Map` of `String`s +is used to configure the `ElasticsearchSink`. This config map will be directly +forwarded when creating the internally used `TransportClient`. 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. +For Elasticsearch 6.x and above, internally, the `RestHighLevelClient` is used for cluster communication. +By default, the connector uses the default configurations for the REST client. To have custom +configuration for the REST client, users can provide a `RestClientFactory` implementation when +setting up the `ElasticsearchClient.Builder` that builds the sink. + Also note that the example only demonstrates performing a single index request for each incoming element. Generally, the `ElasticsearchSinkFunction` can be used to perform multiple requests of different types (ex., From 6ae0340d41ea6a6fa67b24dcdf1aa71eb07f879d Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 17:09:46 +0800 Subject: [PATCH 11/21] fixup! Add imports to doc code snippets --- docs/dev/connectors/elasticsearch.md | 94 ++++++++++++++++++++++++++++ 1 file changed, 94 insertions(+) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index eb4f74f6b805e..42a8cacf1ccd5 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -84,6 +84,23 @@ The example below shows how to configure and create a sink:
{% highlight java %} +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink; +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; + DataStream input = ...; Map config = new HashMap<>(); @@ -115,6 +132,22 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea
{% highlight java %} +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStream; +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.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + DataStream input = ...; Map config = new HashMap<>(); @@ -145,6 +178,20 @@ input.addSink(new ElasticsearchSink<>(config, transportAddresses, new Elasticsea
{% highlight java %} +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer; +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink; + +import org.apache.http.HttpHost; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.client.Requests; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + DataStream input = ...; List httpHost = new ArrayList<>(); @@ -195,6 +242,23 @@ input.addSink(esSinkBuilder.build());
{% highlight scala %} +import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSink +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 + val input: DataStream[String] = ... val config = new java.util.HashMap[String, String] @@ -221,6 +285,22 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc
{% highlight scala %} +import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.streaming.api.datastream.DataStream +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.elasticsearch.action.index.IndexRequest +import org.elasticsearch.client.Requests + +import java.net.InetAddress +import java.net.InetSocketAddress +import java.util.ArrayList +import java.util.HashMap +import java.util.List +import java.util.Map + val input: DataStream[String] = ... val config = new java.util.HashMap[String, String] @@ -247,6 +327,20 @@ input.addSink(new ElasticsearchSink(config, transportAddresses, new Elasticsearc
{% highlight scala %} +import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.connectors.elasticsearch.RequestIndexer +import org.apache.flink.streaming.connectors.elasticsearch6.ElasticsearchSink + +import org.apache.http.HttpHost +import org.elasticsearch.action.index.IndexRequest +import org.elasticsearch.client.Requests + +import java.util.ArrayList +import java.util.HashMap +import java.util.List +import java.util.Map + val input: DataStream[String] = ... val httpHosts = new java.util.ArrayList[HttpHost] From 57dd9aef3beb02157b6e0f16f2db3248460f084a Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Fri, 27 Jul 2018 17:28:42 +0800 Subject: [PATCH 12/21] fixup! More checkstyles --- .../connectors/elasticsearch6/ElasticsearchSink.java | 7 ++++++- .../connectors/elasticsearch6/RestClientFactory.java | 1 + 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index c60ac4d922062..0ae24e03161fa 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -22,8 +22,8 @@ import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase; import org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkFunction; import org.apache.flink.streaming.connectors.elasticsearch.util.NoOpFailureHandler; - import org.apache.flink.util.Preconditions; + import org.apache.http.HttpHost; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.bulk.BulkProcessor; @@ -80,6 +80,11 @@ private ElasticsearchSink( super(new Elasticsearch6ApiCallBridge(httpHosts, restClientFactory), bulkRequestsConfig, elasticsearchSinkFunction, failureHandler); } + /** + * A builder for creating an {@link ElasticsearchSink}. + * + * @param Type of the elements handled by the sink this builder creates. + */ @PublicEvolving public static class Builder { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java index 82673cd644a54..4b74649ca8718 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/RestClientFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch6; import org.apache.flink.annotation.PublicEvolving; + import org.elasticsearch.client.RestClientBuilder; import java.io.Serializable; From 812d6e265b319e5ad70359bc3f77c441677ff4bd Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 30 Jul 2018 15:51:57 +0800 Subject: [PATCH 13/21] fixup! Fix log4j dependency versions --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index e453837fbc996..9032fcbfb52e0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -141,14 +141,14 @@ under the License. org.apache.logging.log4j log4j-api - 2.7 + 2.9.1 test org.apache.logging.log4j log4j-core - 2.7 + 2.9.1 test From c392ab86d3e30e93ed1c1e2b411a43e8416b37dc Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 30 Jul 2018 17:35:00 +0800 Subject: [PATCH 14/21] fixip! Fix failing ITCases for Elasticsearch 6.x --- .../ElasticsearchApiCallBridge.java | 3 +- .../ElasticsearchSinkTestBase.java | 28 ++++++++++++------- .../ElasticsearchSinkITCase.java | 17 +++++++++++ .../ElasticsearchSinkITCase.java | 11 +++++++- .../ElasticsearchSinkITCase.java | 12 +++++++- .../Elasticsearch6ApiCallBridge.java | 7 ++++- .../ElasticsearchSinkITCase.java | 11 +++++++- 7 files changed, 74 insertions(+), 15 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java index 650f4c640e30f..f1dcc83f652fa 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchApiCallBridge.java @@ -25,6 +25,7 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.io.Serializable; import java.util.Map; @@ -48,7 +49,7 @@ public interface ElasticsearchApiCallBridge extends Ser * @param clientConfig The configuration to use when constructing the client. * @return The created client. */ - C createClient(Map clientConfig); + C createClient(Map clientConfig) throws IOException; /** * Creates a {@link BulkProcessor.Builder} for creating the bulk processor. diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index fd8a54c053ac8..1ba177d35a159 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -88,7 +88,7 @@ public static void shutdown() throws Exception { * Tests that the Elasticsearch sink works properly. */ public void runElasticsearchSinkTest() throws Exception { - final String index = "transport-client-test-index"; + final String index = "elasticsearch-sink-test-index"; final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); @@ -102,7 +102,7 @@ public void runElasticsearchSinkTest() throws Exception { source.addSink(createElasticsearchSinkForEmbeddedNode( userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); - env.execute("Elasticsearch TransportClient Test"); + env.execute("Elasticsearch Sink Test"); // verify the results Client client = embeddedNodeEnv.getClient(); @@ -117,11 +117,11 @@ public void runElasticsearchSinkTest() throws Exception { public void runNullAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", CLUSTER_NAME); try { createElasticsearchSink(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); - } catch (IllegalArgumentException expectedException) { + } catch (IllegalArgumentException | NullPointerException expectedException) { // test passes return; } @@ -135,7 +135,7 @@ public void runNullAddressesTest() throws Exception { public void runEmptyAddressesTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", CLUSTER_NAME); try { createElasticsearchSink( @@ -160,15 +160,17 @@ public void runInvalidElasticsearchClusterTest() throws Exception { Map userConfig = new HashMap<>(); userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", "my-transport-client-cluster"); + userConfig.put("cluster.name", "invalid-cluster-name"); - source.addSink(createElasticsearchSinkForEmbeddedNode( - Collections.unmodifiableMap(userConfig), new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"))); + source.addSink(createElasticsearchSinkForNode( + Collections.unmodifiableMap(userConfig), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"), + "123.123.123.123")); // incorrect ip address try { - env.execute("Elasticsearch Transport Client Test"); + env.execute("Elasticsearch Sink Test"); } catch (JobExecutionException expectedException) { - expectedException.printStackTrace(); + // test passes return; } @@ -189,4 +191,10 @@ protected abstract ElasticsearchSinkBase, C> createElast */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; + + /** + * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. + */ + protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForNode( + Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception; } 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 c3e9b01bfac96..1ac53aa79da91 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 @@ -28,10 +28,12 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; +import org.elasticsearch.common.transport.InetSocketTransportAddress; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.junit.Test; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collections; @@ -128,6 +130,21 @@ protected ElasticsearchSinkBase, Client> createElasticse elasticsearchSinkFunction); } + @Override + protected ElasticsearchSinkBase, Client> createElasticsearchSinkForNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + + List transports = new ArrayList<>(); + transports.add(new InetSocketTransportAddress(InetAddress.getByName(ipAddress), 9300)); + + return new ElasticsearchSink<>( + Collections.unmodifiableMap(userConfig), + transports, + elasticsearchSinkFunction); + } + /** * A {@link IndexRequestBuilder} with equivalent functionality to {@link SourceSinkDataTestKit.TestElasticsearchSinkFunction}. */ 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 9d996ad212e9a..75584654fd6e4 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 @@ -70,8 +70,17 @@ protected ElasticsearchSinkBase, TransportClient> create Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + } + + @Override + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + List transports = new ArrayList<>(); - transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); + transports.add(new InetSocketAddress(InetAddress.getByName(ipAddress), 9300)); return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction); } 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 f7ed056706b6c..127eff5beccff 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 @@ -70,8 +70,18 @@ protected ElasticsearchSinkBase, TransportClient> create protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForEmbeddedNode( Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + + return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + } + + @Override + protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + List transports = new ArrayList<>(); - transports.add(new InetSocketAddress(InetAddress.getByName("127.0.0.1"), 9300)); + transports.add(new InetSocketAddress(InetAddress.getByName(ipAddress), 9300)); return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index 1912134171b5f..ea3732f599180 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -35,6 +35,7 @@ import javax.annotation.Nullable; +import java.io.IOException; import java.util.List; import java.util.Map; @@ -65,12 +66,16 @@ public class Elasticsearch6ApiCallBridge implements ElasticsearchApiCallBridge clientConfig) { + public RestHighLevelClient createClient(Map clientConfig) throws IOException { RestClientBuilder builder = RestClient.builder(httpHosts.toArray(new HttpHost[httpHosts.size()])); restClientFactory.configureRestClientBuilder(builder); RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + if (!rhlClient.ping()) { + throw new RuntimeException("There are no reachable Elasticsearch nodes!"); + } + if (LOG.isInfoEnabled()) { LOG.info("Created Elasticsearch RestHighLevelClient connected to {}", httpHosts.toString()); } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 8c456143d6a8b..3c2a96431f8b3 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -73,8 +73,17 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + } + + @Override + protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception { + ArrayList httpHosts = new ArrayList<>(); - httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); + httpHosts.add(new HttpHost(ipAddress, 9200, "http")); ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); builder.setBulkRequestsConfig(userConfig); From f0bf3d5e7d162f5afa88060f44d08476a86de140 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 31 Jul 2018 17:32:55 +0800 Subject: [PATCH 15/21] fixup! Address last review comments --- docs/dev/connectors/elasticsearch.md | 16 +--- .../elasticsearch/ElasticsearchSinkBase.java | 6 +- .../elasticsearch6/ElasticsearchSink.java | 93 ++++++++++++++++--- .../test-scripts/elasticsearch-common.sh | 7 +- .../test_streaming_elasticsearch.sh | 3 - 5 files changed, 95 insertions(+), 30 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 42a8cacf1ccd5..68cf280cfad5e 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -198,10 +198,6 @@ List httpHost = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); httpHosts.add(new HttpHost("10.2.3.1", 9200, "http")); -Map bulkRequestsConfig = new HashMap<>(); -// This instructs the sink to emit after every element, otherwise they would be buffered -bulkRequestsConfig.put("bulk.flush.max.actions", "1"); - // use a ElasticsearchSink.Builder to create an ElasticsearchSink ElasticsearchSink.Builder esSinkBuilder = new ElasticsearchSink.Builder<>( httpHosts, @@ -223,8 +219,8 @@ ElasticsearchSink.Builder esSinkBuilder = new ElasticsearchSink.Builder< } ); -// configuration for the bulk requests -builder.setBulkRequestsConfig(bulkRequestsConfig); +// configuration for the bulk requests; this instructs the sink to emit after every element, otherwise they would be buffered +builder.setBulkFlushMaxActions(1); // provide a RestClientFactory for custom configuration on the internally created REST client builder.setRestClientBuilder( @@ -347,10 +343,6 @@ val httpHosts = new java.util.ArrayList[HttpHost] httpHosts.add(new HttpHost("127.0.0.1", 9300, "http")) httpHosts.add(new HttpHost("10.2.3.1", 9300, "http")) -val bulkRequestsConfig = new java.util.HashMap[String, String] -// This instructs the sink to emit after every element, otherwise they would be buffered -bulkRequestsConfig.put("bulk.flush.max.actions", "1") - val esSinkBuilder = new ElasticsearchSink.Builer[String]( httpHosts, new ElasticsearchSinkFunction[String] { @@ -366,8 +358,8 @@ val esSinkBuilder = new ElasticsearchSink.Builer[String]( } ) -// configuration for the bulk requests -builder.setBulkRequestsConfig(bulkRequestsConfig) +// configuration for the bulk requests; this instructs the sink to emit after every element, otherwise they would be buffered +builder.setBulkFlushMaxActions(1) // provide a RestClientFactory for custom configuration on the internally created REST client builder.setRestClientBuilder( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java index 1c5b1f3d60953..7dac06ceb8a7a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.connectors.elasticsearch; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.Configuration; @@ -86,6 +87,7 @@ public abstract class ElasticsearchSinkBase extends /** * Used to control whether the retry delay should increase exponentially or remain constant. */ + @PublicEvolving public enum FlushBackoffType { CONSTANT, EXPONENTIAL @@ -136,7 +138,7 @@ public void setDelayMillis(long delayMillis) { private final Integer bulkProcessorFlushMaxActions; private final Integer bulkProcessorFlushMaxSizeMb; - private final Integer bulkProcessorFlushIntervalMillis; + private final Long bulkProcessorFlushIntervalMillis; private final BulkFlushBackoffPolicy bulkProcessorFlushBackoffPolicy; // ------------------------------------------------------------------------ @@ -244,7 +246,7 @@ public ElasticsearchSinkBase( } if (params.has(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS)) { - bulkProcessorFlushIntervalMillis = params.getInt(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); + bulkProcessorFlushIntervalMillis = params.getLong(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); userConfig.remove(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS); } else { bulkProcessorFlushIntervalMillis = null; diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 0ae24e03161fa..ac803cd8cabc1 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -61,15 +61,6 @@ public class ElasticsearchSink extends ElasticsearchSinkBase bulkRequestsConfig, List httpHosts, @@ -95,23 +86,103 @@ public static class Builder { private ActionRequestFailureHandler failureHandler = new NoOpFailureHandler(); private RestClientFactory restClientFactory = restClientBuilder -> {}; + /** + * Creates a new {@code ElasticsearchSink} that connects to the cluster using a {@link RestHighLevelClient}. + * + * @param httpHosts The list of {@link HttpHost} to which the {@link RestHighLevelClient} connects to. + * @param elasticsearchSinkFunction This is used to generate multiple {@link ActionRequest} from the incoming element. + */ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsearchSinkFunction) { this.httpHosts = Preconditions.checkNotNull(httpHosts); this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); } - public void setBulkRequestsConfig(Map bulkRequestsConfig) { - this.bulkRequestsConfig = bulkRequestsConfig; + /** + * Sets the maximum number of actions to buffer for each bulk request. + * + * @param numMaxActions the maxinum number of actions to buffer per bulk request. + */ + public void setBulkFlushMaxActions(int numMaxActions) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); + } + + /** + * Sets the maximum size of buffered actions, in mb, per bulk request. + * + * @param maxSizeMb the maximum size of buffered actions, in mb. + */ + public void setBulkFlushMaxSizeMb(int maxSizeMb) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); + } + + /** + * Sets the bulk flush interval, in milliseconds. + * + * @param intervalMillis the bulk flush interval, in milliseconds. + */ + public void setBulkFlushInterval(long intervalMillis) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); + } + + /** + * Sets whether or not to enable bulk flush backoff behaviour. + * + * @param enabled whether or not to enable backoffs. + */ + public void setBulkFlushBackoff(boolean enabled) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE, String.valueOf(enabled)); + } + + /** + * Sets the type of back of to use when flushing bulk requests. + * + * @param flushBackoffType the backoff type to use. + */ + public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, flushBackoffType.toString()); + } + + /** + * Sets the maximum number of retries for a backoff attempt when flushing bulk requests. + * + * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests + */ + public void setBulkFlushBackoffRetries(int maxRetries) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); + } + + /** + * Sets the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + * + * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. + */ + public void setBulkFlushBackoffDelay(long delayMillis) { + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); } + /** + * Sets a failure handler for action requests. + * + * @param failureHandler This is used to handle failed {@link ActionRequest}. + */ public void setFailureHandler(ActionRequestFailureHandler failureHandler) { this.failureHandler = failureHandler; } + /** + * Sets a REST client factory for custom client configuration. + * + * @param restClientFactory the factory that configures the rest client. + */ public void setRestClientFactory(RestClientFactory restClientFactory) { this.restClientFactory = restClientFactory; } + /** + * Creates the Elasticsearch sink. + * + * @return the created Elasticsearch sink. + */ public ElasticsearchSink build() { return new ElasticsearchSink<>(bulkRequestsConfig, httpHosts, elasticsearchSinkFunction, failureHandler, restClientFactory); } diff --git a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh index 900b73664c26a..fa6c33124be92 100644 --- a/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh +++ b/flink-end-to-end-tests/test-scripts/elasticsearch-common.sh @@ -42,7 +42,7 @@ function setup_elasticsearch { } function verify_elasticsearch_process_exist { - while : ; do + for ((i=1;i<=10;i++)); do local elasticsearchProcess=$(jps | grep Elasticsearch | awk '{print $2}') echo "Waiting for Elasticsearch node to start ..." @@ -52,9 +52,12 @@ function verify_elasticsearch_process_exist { sleep 1 else echo "Elasticsearch node is running." - break + return fi done + + echo "Elasticsearch node did not start properly" + exit 1 } function verify_result { diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh b/flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh index 7464409f41e5b..c8cd2db17c951 100755 --- a/flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh +++ b/flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh @@ -32,9 +32,6 @@ start_cluster function test_cleanup { shutdown_elasticsearch_cluster index - - # make sure to run regular cleanup as well - cleanup } trap test_cleanup INT From 8ed06a2417f211b8cd29521bb1ed8855344b3ed9 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 31 Jul 2018 17:39:50 +0800 Subject: [PATCH 16/21] fixup! Add elasticsearch6 module to Travis connector builds --- tools/travis_mvn_watchdog.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index c4620c837af18..ae5e58c2f7673 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -88,6 +88,7 @@ flink-connectors/flink-connector-cassandra,\ flink-connectors/flink-connector-elasticsearch,\ flink-connectors/flink-connector-elasticsearch2,\ flink-connectors/flink-connector-elasticsearch5,\ +flink-connectors/flink-connector-elasticsearch6,\ flink-connectors/flink-connector-elasticsearch-base,\ flink-connectors/flink-connector-filesystem,\ flink-connectors/flink-connector-kafka-0.8,\ From 978201800519df3b847b606dba31fcd757539b7a Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 31 Jul 2018 17:46:38 +0800 Subject: [PATCH 17/21] fixup! fixup! Address last review comments --- .../connectors/elasticsearch6/ElasticsearchSink.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index ac803cd8cabc1..8a15d465eca82 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -97,6 +97,15 @@ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsea this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); } + /** + * Sets a map of strings that configures the bulk flushing behaviour. + * + * @param bulkRequestsConfig user configuration to configure bulk flushing behaviour. + */ + public void setBulkRequestsConfig(Map bulkRequestsConfig) { + this.bulkRequestsConfig = bulkRequestsConfig; + } + /** * Sets the maximum number of actions to buffer for each bulk request. * From 5874731f9ba2e4245da5d7dfdf206ace836a7ba0 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 31 Jul 2018 17:46:59 +0800 Subject: [PATCH 18/21] fixup! Incorrect logf4-to-slf4j version --- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 9032fcbfb52e0..ef06d80512b0f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -81,7 +81,7 @@ under the License. org.apache.logging.log4j log4j-to-slf4j - 2.7 + 2.9.1 From 3d5af6da9ae3c41f4748bcea85983d0661bc8efa Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 1 Aug 2018 13:52:49 +0800 Subject: [PATCH 19/21] fixup! Address comments --- docs/dev/connectors/elasticsearch.md | 2 -- .../elasticsearch/ElasticsearchSinkTestBase.java | 7 +++++-- .../elasticsearch6/Elasticsearch6ApiCallBridge.java | 4 ++++ 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/docs/dev/connectors/elasticsearch.md b/docs/dev/connectors/elasticsearch.md index 68cf280cfad5e..bafe391850b5f 100644 --- a/docs/dev/connectors/elasticsearch.md +++ b/docs/dev/connectors/elasticsearch.md @@ -333,9 +333,7 @@ import org.elasticsearch.action.index.IndexRequest import org.elasticsearch.client.Requests import java.util.ArrayList -import java.util.HashMap import java.util.List -import java.util.Map val input: DataStream[String] = ... diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 1ba177d35a159..3bafdbe2180c8 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -190,11 +190,14 @@ protected abstract ElasticsearchSinkBase, C> createElast * because the Elasticsearch Java API to do so is incompatible across different versions. */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( - Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; /** * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForNode( - Map userConfig, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception; + Map userConfig, + ElasticsearchSinkFunction> elasticsearchSinkFunction, + String ipAddress) throws Exception; } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java index ea3732f599180..03bf9c0710995 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/Elasticsearch6ApiCallBridge.java @@ -72,6 +72,10 @@ public RestHighLevelClient createClient(Map clientConfig) throws RestHighLevelClient rhlClient = new RestHighLevelClient(builder); + if (LOG.isInfoEnabled()) { + LOG.info("Pinging Elasticsearch cluster via hosts {} ...", httpHosts); + } + if (!rhlClient.ping()) { throw new RuntimeException("There are no reachable Elasticsearch nodes!"); } From 38ede501b1129a2b2c7348b0ba4e9bd852227b38 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 1 Aug 2018 14:15:41 +0800 Subject: [PATCH 20/21] fixup! Remove config map variant from builder --- .../ElasticsearchSinkTestBase.java | 51 ++++++++++++------- .../ElasticsearchSinkITCase.java | 20 +++++--- .../ElasticsearchSinkITCase.java | 24 ++++++--- .../ElasticsearchSinkITCase.java | 24 ++++++--- .../elasticsearch6/ElasticsearchSink.java | 36 ++++++++----- .../ElasticsearchSinkITCase.java | 17 ++++--- .../tests/Elasticsearch6SinkExample.java | 8 ++- 7 files changed, 119 insertions(+), 61 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java index 3bafdbe2180c8..819ffba5d2a26 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkTestBase.java @@ -94,13 +94,10 @@ public void runElasticsearchSinkTest() throws Exception { DataStreamSource> source = env.addSource(new SourceSinkDataTestKit.TestDataSourceFunction()); - Map userConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - userConfig.put("cluster.name", CLUSTER_NAME); - source.addSink(createElasticsearchSinkForEmbeddedNode( - userConfig, new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); + 1, + CLUSTER_NAME, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction(index))); env.execute("Elasticsearch Sink Test"); @@ -120,7 +117,11 @@ public void runNullAddressesTest() throws Exception { userConfig.put("cluster.name", CLUSTER_NAME); try { - createElasticsearchSink(userConfig, null, new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + createElasticsearchSink( + 1, + CLUSTER_NAME, + null, + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); } catch (IllegalArgumentException | NullPointerException expectedException) { // test passes return; @@ -139,9 +140,10 @@ public void runEmptyAddressesTest() throws Exception { try { createElasticsearchSink( - userConfig, - Collections.emptyList(), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); + 1, + CLUSTER_NAME, + Collections.emptyList(), + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test")); } catch (IllegalArgumentException expectedException) { // test passes return; @@ -163,9 +165,10 @@ public void runInvalidElasticsearchClusterTest() throws Exception { userConfig.put("cluster.name", "invalid-cluster-name"); source.addSink(createElasticsearchSinkForNode( - Collections.unmodifiableMap(userConfig), - new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"), - "123.123.123.123")); // incorrect ip address + 1, + "invalid-cluster-name", + new SourceSinkDataTestKit.TestElasticsearchSinkFunction("test"), + "123.123.123.123")); // incorrect ip address try { env.execute("Elasticsearch Sink Test"); @@ -177,27 +180,41 @@ public void runInvalidElasticsearchClusterTest() throws Exception { fail(); } + /** + * Utility method to create a user config map. + */ + protected Map createUserConfig(int bulkFlushMaxActions, String clusterName) { + Map userConfig = new HashMap<>(); + userConfig.put("cluster.name", clusterName); + userConfig.put(ElasticsearchSinkBase.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(bulkFlushMaxActions)); + + return userConfig; + } + /** Creates a version-specific Elasticsearch sink, using arbitrary transport addresses. */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSink( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, List addresses, ElasticsearchSinkFunction> elasticsearchSinkFunction); /** * Creates a version-specific Elasticsearch sink to connect to a local embedded Elasticsearch node. * - *

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

This case is singled out from {@link ElasticsearchSinkTestBase#createElasticsearchSink(int, String, List, ElasticsearchSinkFunction)} * because the Elasticsearch Java API to do so is incompatible across different versions. */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForEmbeddedNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception; /** * Creates a version-specific Elasticsearch sink to connect to a specific Elasticsearch node. */ protected abstract ElasticsearchSinkBase, C> createElasticsearchSinkForNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception; } 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 1ac53aa79da91..2f1a65c58eec7 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 @@ -105,20 +105,27 @@ public void testDeprecatedIndexRequestBuilderVariant() throws Exception { @Override protected ElasticsearchSinkBase, Client> createElasticsearchSink( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, List transportAddresses, ElasticsearchSinkFunction> elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, ElasticsearchUtils.convertInetSocketAddresses(transportAddresses), elasticsearchSinkFunction); + + return new ElasticsearchSink<>( + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), + ElasticsearchUtils.convertInetSocketAddresses(transportAddresses), + elasticsearchSinkFunction); } @Override protected ElasticsearchSinkBase, Client> createElasticsearchSinkForEmbeddedNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { + Map userConfig = createUserConfig(bulkFlushMaxActions, clusterName); + // Elasticsearch 1.x requires this setting when using // LocalTransportAddress to connect to a local embedded node - userConfig = new HashMap<>(userConfig); userConfig.put("node.local", "true"); List transports = new ArrayList<>(); @@ -132,7 +139,8 @@ protected ElasticsearchSinkBase, Client> createElasticse @Override protected ElasticsearchSinkBase, Client> createElasticsearchSinkForNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception { @@ -140,7 +148,7 @@ protected ElasticsearchSinkBase, Client> createElasticse transports.add(new InetSocketTransportAddress(InetAddress.getByName(ipAddress), 9300)); return new ElasticsearchSink<>( - Collections.unmodifiableMap(userConfig), + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), transports, elasticsearchSinkFunction); } 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 75584654fd6e4..7887e72fa1084 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 @@ -28,8 +28,8 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import java.util.Map; /** * IT cases for the {@link ElasticsearchSink}. @@ -58,30 +58,40 @@ public void testInvalidElasticsearchCluster() throws Exception{ @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSink( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, List transportAddresses, ElasticsearchSinkFunction> elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, transportAddresses, elasticsearchSinkFunction); + return new ElasticsearchSink<>( + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), + transportAddresses, + elasticsearchSinkFunction); } @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForEmbeddedNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { - return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); } @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception { List transports = new ArrayList<>(); transports.add(new InetSocketAddress(InetAddress.getByName(ipAddress), 9300)); - return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction); + return new ElasticsearchSink<>( + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), + transports, + elasticsearchSinkFunction); } } 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 127eff5beccff..e86d06768db07 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 @@ -29,8 +29,8 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collections; import java.util.List; -import java.util.Map; /** * IT cases for the {@link ElasticsearchSink}. @@ -59,30 +59,40 @@ public void testInvalidElasticsearchCluster() throws Exception{ @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSink( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, List addresses, ElasticsearchSinkFunction> elasticsearchSinkFunction) { - return new ElasticsearchSink<>(userConfig, addresses, elasticsearchSinkFunction); + return new ElasticsearchSink<>( + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), + addresses, + elasticsearchSinkFunction); } @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForEmbeddedNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { - return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); } @Override protected ElasticsearchSinkBase, TransportClient> createElasticsearchSinkForNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception { List transports = new ArrayList<>(); transports.add(new InetSocketAddress(InetAddress.getByName(ipAddress), 9300)); - return new ElasticsearchSink<>(userConfig, transports, elasticsearchSinkFunction); + return new ElasticsearchSink<>( + Collections.unmodifiableMap(createUserConfig(bulkFlushMaxActions, clusterName)), + transports, + elasticsearchSinkFunction); } } diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java index 8a15d465eca82..4e7a2635738f0 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSink.java @@ -97,21 +97,16 @@ public Builder(List httpHosts, ElasticsearchSinkFunction elasticsea this.elasticsearchSinkFunction = Preconditions.checkNotNull(elasticsearchSinkFunction); } - /** - * Sets a map of strings that configures the bulk flushing behaviour. - * - * @param bulkRequestsConfig user configuration to configure bulk flushing behaviour. - */ - public void setBulkRequestsConfig(Map bulkRequestsConfig) { - this.bulkRequestsConfig = bulkRequestsConfig; - } - /** * Sets the maximum number of actions to buffer for each bulk request. * * @param numMaxActions the maxinum number of actions to buffer per bulk request. */ public void setBulkFlushMaxActions(int numMaxActions) { + Preconditions.checkArgument( + numMaxActions > 0, + "Max number of buffered actions must be larger than 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, String.valueOf(numMaxActions)); } @@ -121,6 +116,10 @@ public void setBulkFlushMaxActions(int numMaxActions) { * @param maxSizeMb the maximum size of buffered actions, in mb. */ public void setBulkFlushMaxSizeMb(int maxSizeMb) { + Preconditions.checkArgument( + maxSizeMb > 0, + "Max size of buffered actions must be larger than 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB, String.valueOf(maxSizeMb)); } @@ -130,6 +129,10 @@ public void setBulkFlushMaxSizeMb(int maxSizeMb) { * @param intervalMillis the bulk flush interval, in milliseconds. */ public void setBulkFlushInterval(long intervalMillis) { + Preconditions.checkArgument( + intervalMillis >= 0, + "Interval (in milliseconds) between each flush must be larger than or equal to 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_INTERVAL_MS, String.valueOf(intervalMillis)); } @@ -148,7 +151,9 @@ public void setBulkFlushBackoff(boolean enabled) { * @param flushBackoffType the backoff type to use. */ public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { - this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, flushBackoffType.toString()); + this.bulkRequestsConfig.put( + CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE, + Preconditions.checkNotNull(flushBackoffType).toString()); } /** @@ -157,6 +162,10 @@ public void setBulkFlushBackoffType(FlushBackoffType flushBackoffType) { * @param maxRetries the maximum number of retries for a backoff attempt when flushing bulk requests */ public void setBulkFlushBackoffRetries(int maxRetries) { + Preconditions.checkArgument( + maxRetries > 0, + "Max number of backoff attempts must be larger than 0."); + this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES, String.valueOf(maxRetries)); } @@ -166,6 +175,9 @@ public void setBulkFlushBackoffRetries(int maxRetries) { * @param delayMillis the amount of delay between each backoff attempt when flushing bulk requests, in milliseconds. */ public void setBulkFlushBackoffDelay(long delayMillis) { + Preconditions.checkArgument( + delayMillis >= 0, + "Delay (in milliseconds) between each backoff attempt must be larger than or equal to 0."); this.bulkRequestsConfig.put(CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY, String.valueOf(delayMillis)); } @@ -175,7 +187,7 @@ public void setBulkFlushBackoffDelay(long delayMillis) { * @param failureHandler This is used to handle failed {@link ActionRequest}. */ public void setFailureHandler(ActionRequestFailureHandler failureHandler) { - this.failureHandler = failureHandler; + this.failureHandler = Preconditions.checkNotNull(failureHandler); } /** @@ -184,7 +196,7 @@ public void setFailureHandler(ActionRequestFailureHandler failureHandler) { * @param restClientFactory the factory that configures the rest client. */ public void setRestClientFactory(RestClientFactory restClientFactory) { - this.restClientFactory = restClientFactory; + this.restClientFactory = Preconditions.checkNotNull(restClientFactory); } /** diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 3c2a96431f8b3..6ae77b6e86538 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Map; /** * IT cases for the {@link ElasticsearchSink}. @@ -58,27 +57,31 @@ public void testInvalidElasticsearchCluster() throws Exception{ @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSink( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, List httpHosts, ElasticsearchSinkFunction> elasticsearchSinkFunction) { ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkRequestsConfig(userConfig); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); return builder.build(); } @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForEmbeddedNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction) throws Exception { - return createElasticsearchSinkForNode(userConfig, elasticsearchSinkFunction, "127.0.0.1"); + return createElasticsearchSinkForNode( + bulkFlushMaxActions, clusterName, elasticsearchSinkFunction, "127.0.0.1"); } @Override protected ElasticsearchSinkBase, RestHighLevelClient> createElasticsearchSinkForNode( - Map userConfig, + int bulkFlushMaxActions, + String clusterName, ElasticsearchSinkFunction> elasticsearchSinkFunction, String ipAddress) throws Exception { @@ -86,7 +89,7 @@ protected ElasticsearchSinkBase, RestHighLevelClient> cr httpHosts.add(new HttpHost(ipAddress, 9200, "http")); ElasticsearchSink.Builder> builder = new ElasticsearchSink.Builder<>(httpHosts, elasticsearchSinkFunction); - builder.setBulkRequestsConfig(userConfig); + builder.setBulkFlushMaxActions(bulkFlushMaxActions); return builder.build(); } diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java index 8e0c59f865504..dedcbb28f081a 100644 --- a/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/src/main/java/org/apache/flink/streaming/tests/Elasticsearch6SinkExample.java @@ -61,17 +61,15 @@ public String map(Long value) throws Exception { } }); - Map bulkRequestsConfig = new HashMap<>(); - // This instructs the sink to emit after every element, otherwise they would be buffered - bulkRequestsConfig.put(ElasticsearchSink.CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS, "1"); - List httpHosts = new ArrayList<>(); httpHosts.add(new HttpHost("127.0.0.1", 9200, "http")); ElasticsearchSink.Builder esSinkBuilder = new ElasticsearchSink.Builder<>( httpHosts, (String element, RuntimeContext ctx, RequestIndexer indexer) -> indexer.add(createIndexRequest(element, parameterTool))); - esSinkBuilder.setBulkRequestsConfig(bulkRequestsConfig); + + // this instructs the sink to emit after every element, otherwise they would be buffered + esSinkBuilder.setBulkFlushMaxActions(1); source.addSink(esSinkBuilder.build()); From 70475297219f34d0962cdf6970ac6589da925375 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 1 Aug 2018 14:23:42 +0800 Subject: [PATCH 21/21] fixup! Add comments to 5.x and 6.x ITCases about why tests cannot be executed in IDE --- .../connectors/elasticsearch5/ElasticsearchSinkITCase.java | 4 ++++ .../connectors/elasticsearch6/ElasticsearchSinkITCase.java | 4 ++++ 2 files changed, 8 insertions(+) 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 e86d06768db07..67daa409b7b49 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 @@ -34,6 +34,10 @@ /** * IT cases for the {@link ElasticsearchSink}. + * + *

The Elasticsearch ITCases for 5.x CANNOT be executed in the IDE directly, since it is required that the + * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded + * node used in the tests to work properly. */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase { diff --git a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java index 6ae77b6e86538..a6f0125894094 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/test/java/org/apache/flink/streaming/connectors/elasticsearch6/ElasticsearchSinkITCase.java @@ -32,6 +32,10 @@ /** * IT cases for the {@link ElasticsearchSink}. + * + *

The Elasticsearch ITCases for 6.x CANNOT be executed in the IDE directly, since it is required that the + * Log4J-to-SLF4J adapter dependency must be excluded from the test classpath for the Elasticsearch embedded + * node used in the tests to work properly. */ public class ElasticsearchSinkITCase extends ElasticsearchSinkTestBase {