Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A258C200CBB for ; Tue, 4 Jul 2017 10:31:05 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A1019160BE1; Tue, 4 Jul 2017 08:31:05 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id BD7F9160BEF for ; Tue, 4 Jul 2017 10:31:04 +0200 (CEST) Received: (qmail 88425 invoked by uid 500); 4 Jul 2017 08:31:04 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 88416 invoked by uid 99); 4 Jul 2017 08:31:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Jul 2017 08:31:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 80288D0BDD for ; Tue, 4 Jul 2017 08:31:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.211 X-Spam-Level: X-Spam-Status: No, score=-99.211 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 67TupIINE-CB for ; Tue, 4 Jul 2017 08:31:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 8321260CD3 for ; Tue, 4 Jul 2017 08:31:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 5FE54E0BCB for ; Tue, 4 Jul 2017 08:31:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 11C2B24602 for ; Tue, 4 Jul 2017 08:31:00 +0000 (UTC) Date: Tue, 4 Jul 2017 08:31:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-6996) FlinkKafkaProducer010 doesn't guarantee at-least-once semantic MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 04 Jul 2017 08:31:05 -0000 [ https://issues.apache.org/jira/browse/FLINK-6996?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16073315#comment-16073315 ] ASF GitHub Bot commented on FLINK-6996: --------------------------------------- Github user pnowojski commented on a diff in the pull request: https://github.com/apache/flink/pull/4206#discussion_r125416757 --- Diff: flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java --- @@ -172,6 +195,144 @@ public void cancel() { } } + /** + * Tests the at-least-once semantic for the simple writes into Kafka. + */ + @Test + public void testOneToOneAtLeastOnceRegularSink() throws Exception { + testOneToOneAtLeastOnce(true); + } + + /** + * Tests the at-least-once semantic for the simple writes into Kafka. + */ + @Test + public void testOneToOneAtLeastOnceCustomOperator() throws Exception { + testOneToOneAtLeastOnce(false); + } + + /** + * This test sets KafkaProducer so that it will not automatically flush the data and + * and fails the broker to check whether FlinkKafkaProducer flushed records manually on snapshotState. + */ + protected void testOneToOneAtLeastOnce(boolean regularSink) throws Exception { + final String topic = regularSink ? "oneToOneTopicRegularSink" : "oneToOneTopicCustomOperator"; + final int partition = 0; + final int numElements = 1000; + final int failAfterElements = 333; + + createTestTopic(topic, 1, 1); + + TypeInformationSerializationSchema schema = new TypeInformationSerializationSchema<>(BasicTypeInfo.INT_TYPE_INFO, new ExecutionConfig()); + KeyedSerializationSchema keyedSerializationSchema = new KeyedSerializationSchemaWrapper(schema); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(500); + env.setParallelism(1); + env.setRestartStrategy(RestartStrategies.noRestart()); + env.getConfig().disableSysoutLogging(); + + Properties properties = new Properties(); + properties.putAll(standardProps); + properties.putAll(secureProps); + // decrease timeout and block time from 60s down to 10s - this is how long KafkaProducer will try send pending (not flushed) data on close() + properties.setProperty("timeout.ms", "10000"); + properties.setProperty("max.block.ms", "10000"); + // increase batch.size and linger.ms - this tells KafkaProducer to batch produced events instead of flushing them immediately + properties.setProperty("batch.size", "10240000"); + properties.setProperty("linger.ms", "10000"); + + int leaderId = kafkaServer.getLeaderToShutDown(topic); + BrokerRestartingMapper.resetState(); + + // process exactly failAfterElements number of elements and then shutdown Kafka broker and fail application + DataStream inputStream = env + .fromCollection(getIntegersSequence(numElements)) + .map(new BrokerRestartingMapper(leaderId, failAfterElements)); + + StreamSink kafkaSink = kafkaServer.getProducerSink(topic, keyedSerializationSchema, properties, new FlinkKafkaPartitioner() { + @Override + public int partition(Integer record, byte[] key, byte[] value, String targetTopic, int[] partitions) { + return partition; + } + }); + + if (regularSink) { + inputStream.addSink(kafkaSink.getUserFunction()); + } + else { + kafkaServer.produceIntoKafka(inputStream, topic, keyedSerializationSchema, properties, new FlinkKafkaPartitioner() { + @Override + public int partition(Integer record, byte[] key, byte[] value, String targetTopic, int[] partitions) { + return partition; + } + }); + } + + FailingIdentityMapper.failedBefore = false; + try { + env.execute("One-to-one at least once test"); + fail("Job should fail!"); + } + catch (Exception ex) { --- End diff -- FYI `getCause` Exception is type of `java.lang.Exception`, so there is no point in making an assertion on that. > FlinkKafkaProducer010 doesn't guarantee at-least-once semantic > -------------------------------------------------------------- > > Key: FLINK-6996 > URL: https://issues.apache.org/jira/browse/FLINK-6996 > Project: Flink > Issue Type: Bug > Components: Kafka Connector > Affects Versions: 1.2.0, 1.3.0, 1.2.1, 1.3.1 > Reporter: Piotr Nowojski > Assignee: Piotr Nowojski > > FlinkKafkaProducer010 doesn't implement CheckpointedFunction interface. This means, when it's used like a "regular sink function" (option a from [the java doc|https://ci.apache.org/projects/flink/flink-docs-master/api/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer010.html]) it will not flush the data on "snapshotState" as it is supposed to. -- This message was sent by Atlassian JIRA (v6.4.14#64029)