[ 
https://issues.apache.org/jira/browse/FLINK-5487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15875785#comment-15875785
 ] 

ASF GitHub Bot commented on FLINK-5487:
---------------------------------------

Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3358#discussion_r102178621
  
    --- Diff: 
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java
 ---
    @@ -67,10 +73,56 @@
        public static final String CONFIG_KEY_BULK_FLUSH_MAX_ACTIONS = 
"bulk.flush.max.actions";
        public static final String CONFIG_KEY_BULK_FLUSH_MAX_SIZE_MB = 
"bulk.flush.max.size.mb";
        public static final String CONFIG_KEY_BULK_FLUSH_INTERVAL_MS = 
"bulk.flush.interval.ms";
    +   public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_ENABLE = 
"bulk.flush.backoff.enable";
    +   public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_TYPE = 
"bulk.flush.backoff.type";
    +   public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_RETRIES = 
"bulk.flush.backoff.retries";
    +   public static final String CONFIG_KEY_BULK_FLUSH_BACKOFF_DELAY = 
"bulk.flush.backoff.delay";
    +
    +   public enum FlushBackoffType {
    +           CONSTANT,
    +           EXPONENTIAL
    +   }
    +
    +   public class BulkFlushBackoffPolicy implements Serializable {
    +
    +           private static final long serialVersionUID = 
-6022851996101826049L;
    +
    +           // the default values follow the Elasticsearch default settings 
for BulkProcessor
    +           private FlushBackoffType backoffType = 
FlushBackoffType.EXPONENTIAL;
    +           private int maxRetryCount = 8;
    +           private long delayMillis = 50;
    +
    +           public FlushBackoffType getBackoffType() {
    +                   return backoffType;
    +           }
    +
    +           public int getMaxRetryCount() {
    +                   return maxRetryCount;
    +           }
    +
    +           public long getDelayMillis() {
    +                   return delayMillis;
    +           }
    +
    +           public void setBackoffType(FlushBackoffType backoffType) {
    +                   this.backoffType = checkNotNull(backoffType);
    +           }
    +
    +           public void setMaxRetryCount(int maxRetryCount) {
    +                   checkArgument(maxRetryCount > 0);
    +                   this.maxRetryCount = maxRetryCount;
    +           }
    +
    +           public void setDelayMillis(long delayMillis) {
    +                   checkArgument(delayMillis > 0);
    --- End diff --
    
    We should accept 0 here as well, if users want to retry immediately (for 
whatever reason :) )


> Proper at-least-once support for ElasticsearchSink
> --------------------------------------------------
>
>                 Key: FLINK-5487
>                 URL: https://issues.apache.org/jira/browse/FLINK-5487
>             Project: Flink
>          Issue Type: Bug
>          Components: Streaming Connectors
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Tzu-Li (Gordon) Tai
>            Priority: Critical
>
> Discussion in ML: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Fault-tolerance-guarantees-of-Elasticsearch-sink-in-flink-elasticsearch2-td10982.html
> Currently, the Elasticsearch Sink actually doesn't offer any guarantees for 
> message delivery.
> For proper support of at-least-once, the sink will need to participate in 
> Flink's checkpointing: when snapshotting is triggered at the 
> {{ElasticsearchSink}}, we need to synchronize on the pending ES requests by 
> flushing the internal bulk processor. For temporary ES failures (see 
> FLINK-5122) that may happen on the flush, we should retry them before 
> returning from snapshotting and acking the checkpoint. If there are 
> non-temporary ES failures on the flush, the current snapshot should fail.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to