[
https://issues.apache.org/jira/browse/FLINK-3857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15321989#comment-15321989
]
ASF GitHub Bot commented on FLINK-3857:
---------------------------------------
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/1962#discussion_r66388492
--- Diff:
flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java
---
@@ -153,9 +165,90 @@ public ElasticsearchSink(Map<String, String>
userConfig, List<InetSocketAddress>
*/
@Override
public void open(Configuration configuration) {
+ connect();
+
+ params = ParameterTool.fromMap(userConfig);
+
+ if (params.has(CONFIG_KEY_CONNECTION_RETRIES)) {
+ this.connectionRetries =
params.getInt(CONFIG_KEY_CONNECTION_RETRIES);
+ }
+
+ buildBulkProcessorIndexer(client);
+ }
+
+ @Override
+ public void invoke(T element) {
+ elasticsearchSinkFunction.process(element, getRuntimeContext(),
requestIndexer);
+
+ if (hasFailure.get()) {
--- End diff --
Another problem with this implementation is that we're capturing the
failure of a bulk operation, but only retrying for the currently processed
element.
Would it be simpler to check whether the Elasticsearch client is still
connected to any nodes before `elasticsearchSinkFunction.process(element,
getRuntimeContext(), requestIndexer);` via `client.connectedNodes().size()`? If
not, then we retry establishing connection before processing the element.
Another approach that might be better is to let the BulkProcessor set a
`lostConnection` flag if it gets thrown a ES connection error for a batch, and
we simply check the flag before in invoke() before doing anything else. But
still we will need have a way to handle all incorrectly records in that batch
due to the lost connection.
> Add reconnect attempt to Elasticsearch host
> -------------------------------------------
>
> Key: FLINK-3857
> URL: https://issues.apache.org/jira/browse/FLINK-3857
> Project: Flink
> Issue Type: Improvement
> Components: Streaming Connectors
> Affects Versions: 1.1.0, 1.0.2
> Reporter: Fabian Hueske
> Assignee: Subhobrata Dey
>
> Currently, the connection to the Elasticsearch host is opened in
> {{ElasticsearchSink.open()}}. In case the connection is lost (maybe due to a
> changed DNS entry), the sink fails.
> I propose to catch the Exception for lost connections in the {{invoke()}}
> method and try to re-open the connection for a configurable number of times
> with a certain delay.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)