Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 6271B200BE1 for ; Mon, 19 Dec 2016 16:19:00 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 613D4160B18; Mon, 19 Dec 2016 15:19:00 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B1628160B30 for ; Mon, 19 Dec 2016 16:18:59 +0100 (CET) Received: (qmail 18505 invoked by uid 500); 19 Dec 2016 15:18:58 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 18481 invoked by uid 99); 19 Dec 2016 15:18:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 19 Dec 2016 15:18:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 6C23E2C03E2 for ; Mon, 19 Dec 2016 15:18:58 +0000 (UTC) Date: Mon, 19 Dec 2016 15:18:58 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-5122) Elasticsearch Sink loses documents when cluster has high load MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 19 Dec 2016 15:19:00 -0000 [ https://issues.apache.org/jira/browse/FLINK-5122?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15761430#comment-15761430 ] ASF GitHub Bot commented on FLINK-5122: --------------------------------------- Github user rmetzger commented on a diff in the pull request: https://github.com/apache/flink/pull/2861#discussion_r93048695 --- Diff: flink-streaming-connectors/flink-connector-elasticsearch2/src/main/java/org/apache/flink/streaming/connectors/elasticsearch2/ElasticsearchSink.java --- @@ -186,22 +198,47 @@ public void beforeBulk(long executionId, BulkRequest request) { @Override public void afterBulk(long executionId, BulkRequest request, BulkResponse response) { + boolean allRequestsRepeatable = true; if (response.hasFailures()) { for (BulkItemResponse itemResp : response.getItems()) { if (itemResp.isFailed()) { - LOG.error("Failed to index document in Elasticsearch: " + itemResp.getFailureMessage()); - failureThrowable.compareAndSet(null, new RuntimeException(itemResp.getFailureMessage())); + // Check if index request can be retried + String failureMessageLowercase = itemResp.getFailureMessage().toLowerCase(); + if (checkErrorAndRetryBulk && ( + failureMessageLowercase.contains("timeout") || failureMessageLowercase.contains("timed out") // Generic timeout errors + || failureMessageLowercase.contains("UnavailableShardsException".toLowerCase()) // Shard not available due to rebalancing or node down + || (failureMessageLowercase.contains("data/write/bulk") && failureMessageLowercase.contains("bulk")) // Bulk index queue on node full + ) + ) { + LOG.debug("Retry bulk: {}", itemResp.getFailureMessage()); + reAddBulkRequest(request); --- End diff -- Are you sure that the `BulkRequest` is only added once even if it contains multiple failed `BulkItemResponse`s? > Elasticsearch Sink loses documents when cluster has high load > ------------------------------------------------------------- > > Key: FLINK-5122 > URL: https://issues.apache.org/jira/browse/FLINK-5122 > Project: Flink > Issue Type: Bug > Components: Streaming Connectors > Affects Versions: 1.2.0 > Reporter: static-max > Assignee: static-max > > My cluster had high load and documents got not indexed. This violates the "at least once" semantics in the ES connector. > I gave pressure on my cluster to test Flink, causing new indices to be created and balanced. On those errors the bulk should be tried again instead of being discarded. > Primary shard not active because ES decided to rebalance the index: > 2016-11-15 15:35:16,123 ERROR org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink - Failed to index document in Elasticsearch: UnavailableShardsException[[index-name][3] primary shard is not active Timeout: [1m], request: [BulkShardRequest to [index-name] containing [20] requests]] > Bulk queue on node full (I set queue to a low value to reproduce error): > 22:37:57,702 ERROR org.apache.flink.streaming.connectors.elasticsearch2.ElasticsearchSink - Failed to index document in Elasticsearch: RemoteTransportException[[node1][192.168.1.240:9300][indices:data/write/bulk[s][p]]]; nested: EsRejectedExecutionException[rejected execution of org.elasticsearch.transport.TransportService$4@727e677c on EsThreadPoolExecutor[bulk, queue capacity = 1, org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor@51322d37[Running, pool size = 2, active threads = 2, queued tasks = 1, completed tasks = 2939]]]; > I can try to propose a PR for this. -- This message was sent by Atlassian JIRA (v6.3.4#6332)