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

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_r102545769
  
    --- Diff: 
flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/ElasticsearchSinkBase.java
 ---
    @@ -211,6 +283,23 @@ public void invoke(T value) throws Exception {
        }
     
        @Override
    +   public void initializeState(FunctionInitializationContext context) 
throws Exception {
    +           // no initialization needed
    +   }
    +
    +   @Override
    +   public void snapshotState(FunctionSnapshotContext context) throws 
Exception {
    +           checkErrorAndRethrow();
    +
    +           if (flushOnCheckpoint) {
    +                   do {
    +                           bulkProcessor.flush();
    --- End diff --
    
    This flush() might be a noop if bulkRequest.numberOfActions() == 0 in the 
bulkProcessor implementation.
    If so, this loop turns into a busy loop wasting CPU cycles.
    I wonder if we should wait on the numPendingRequests and notify on it once 
we update it?
    
    (Sorry that I bring this up in the second review)


> 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