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

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

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

    https://github.com/apache/flink/pull/2108#discussion_r69130106
  
    --- Diff: 
flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AtLeastOnceProducerTest.java
 ---
    @@ -107,15 +115,16 @@ public void run() {
                threadB.start();
                // this should block:
                producer.snapshotState(0, 0);
    -           // once all pending callbacks are confirmed, we can set this 
marker to true
    -           markOne.set(true);
    -           for(int i = 0; i < 99; i++) {
    -                   producer.invoke("msg-" + i);
    +           synchronized (threadA) {
    +                   threadA.notifyAll(); // just in case, to let the test 
fail faster
                }
    -           // wait at most one second
    -           threadB.join(800L);
    -           Assert.assertFalse("Thread A reached this point too fast", 
threadB.isAlive());
    -           if(runnableError.f0 != null) {
    +
    +           Deadline deadline = FiniteDuration.apply(5, "s").fromNow();
    +           while (deadline.hasTimeLeft() && threadB.isAlive()) {
    +                   threadB.join(500);
    +           }
    +           Assert.assertFalse("Thread A is expected to be finished at this 
point. If not, the test is prone to fail", threadB.isAlive());
    +           if (runnableError.f0 != null) {
                        runnableError.f0.printStackTrace();
                        Assert.fail("Error from thread B: " + runnableError.f0 
);
    --- End diff --
    
    Printing the stack trace to stdout is imo not so good. The problem is that 
the stack trace will be intermingled with the rest of the testing log output. I 
think it's better to simply rethrow the `Throwable` here.


> FlinkKafkaProducer09 sink can lose messages
> -------------------------------------------
>
>                 Key: FLINK-4027
>                 URL: https://issues.apache.org/jira/browse/FLINK-4027
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.0.3
>            Reporter: Elias Levy
>            Assignee: Robert Metzger
>            Priority: Critical
>
> The FlinkKafkaProducer09 sink appears to not offer at-least-once guarantees.
> The producer is publishing messages asynchronously.  A callback can record 
> publishing errors, which will be raised when detected.  But as far as I can 
> tell, there is no barrier to wait for async errors from the sink when 
> checkpointing or to track the event time of acked messages to inform the 
> checkpointing process.
> If a checkpoint occurs while there are pending publish requests, and the 
> requests return a failure after the checkpoint occurred, those message will 
> be lost as the checkpoint will consider them processed by the sink.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to