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

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

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

    https://github.com/apache/flink/pull/2108#discussion_r67318905
  
    --- Diff: 
flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/AtLeastOnceProducerTest.java
 ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.connectors.kafka;
    +
    +import org.apache.flink.api.java.tuple.Tuple1;
    +import org.apache.flink.configuration.Configuration;
    +import 
org.apache.flink.streaming.connectors.kafka.testutils.MockRuntimeContext;
    +import 
org.apache.flink.streaming.util.serialization.KeyedSerializationSchema;
    +import 
org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
    +import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
    +import org.apache.kafka.clients.producer.Callback;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.Producer;
    +import org.apache.kafka.clients.producer.ProducerRecord;
    +import org.apache.kafka.clients.producer.RecordMetadata;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.serialization.ByteArraySerializer;
    +import org.junit.Assert;
    +import org.junit.Test;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +
    +/**
    + * Test ensuring that the producer is not dropping buffered records
    + */
    +@SuppressWarnings("unchecked")
    +public class AtLeastOnceProducerTest {
    +
    +   @Test
    +   public void testAtLeastOnceProducer() throws Exception {
    +           runTest(true);
    +   }
    +
    +   // This test ensures that the actual test fails if the flushing is 
disabled
    +   @Test(expected = AssertionError.class)
    +   public void ensureTestFails() throws Exception {
    +           runTest(false);
    +   }
    +
    +   private void runTest(boolean flushOnCheckpoint) throws Exception {
    +           Properties props = new Properties();
    +           final TestingKafkaProducer<String> producer = new 
TestingKafkaProducer<>("someTopic", new KeyedSerializationSchemaWrapper<>(new 
SimpleStringSchema()), props);
    +           producer.setFlushOnCheckpoint(flushOnCheckpoint);
    +           producer.setRuntimeContext(new MockRuntimeContext(0, 1));
    +
    +           producer.open(new Configuration());
    +
    +           for(int i = 0; i < 100; i++) {
    --- End diff --
    
    missing space after for


> 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