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

    https://github.com/apache/flink/pull/3358#discussion_r102662136
  
    --- 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 --
    
    Following my arguments above, I think the busy loop you mentioned shouldn't 
happen, because bulk processor's internal `bulkRequest.numberOfActions()` 
should always be synced with our `numPendingRecords`. (i.e., it should not 
occur that `bulkRequest.numberOfActions() == 0` but our own `numPendingRecords 
!= 0`).
    
    So in that case, if `bulkRequest.numberOfActions() == 0` then my original 
loop implementation just fallbacks to a single pass with 2 condition checks.
    
    To a certain extent, I think it might be better to stick to the original 
loop implementation, so that we're not locked-in with how the `BulkProcessor`'s 
flush is implemented. As you can see from a commit I just pushed (2956f99) 
which modifies the mock bulk processor in tests to correctly mimic the flushing 
behaviour I described above, the loop implementation still pass the tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to