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 05DB4200CB0 for ; Fri, 23 Jun 2017 22:52:05 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0467E160BE2; Fri, 23 Jun 2017 20:52: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 4B777160BE5 for ; Fri, 23 Jun 2017 22:52:04 +0200 (CEST) Received: (qmail 44821 invoked by uid 500); 23 Jun 2017 20:52:03 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 44810 invoked by uid 99); 23 Jun 2017 20:52:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 23 Jun 2017 20:52:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 032591AFD4B for ; Fri, 23 Jun 2017 20:52:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id dHNju9d7jf-t for ; Fri, 23 Jun 2017 20:52: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 23B985FC12 for ; Fri, 23 Jun 2017 20:52: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 561DDE0041 for ; Fri, 23 Jun 2017 20:52: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 0C01F2193E for ; Fri, 23 Jun 2017 20:52:00 +0000 (UTC) Date: Fri, 23 Jun 2017 20:52:00 +0000 (UTC) From: "Apurva Mehta (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Assigned] (KAFKA-5494) Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 23 Jun 2017 20:52:05 -0000 [ https://issues.apache.org/jira/browse/KAFKA-5494?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Apurva Mehta reassigned KAFKA-5494: ----------------------------------- Assignee: Apurva Mehta > Idempotent producer should not require max.in.flight.requests.per.connection=1 and acks=all > ------------------------------------------------------------------------------------------- > > Key: KAFKA-5494 > URL: https://issues.apache.org/jira/browse/KAFKA-5494 > Project: Kafka > Issue Type: Sub-task > Affects Versions: 0.11.0.0 > Reporter: Apurva Mehta > Assignee: Apurva Mehta > Labels: exactly-once > > Currently, the idempotent producer (and hence transactional producer) requires max.in.flight.requests.per.connection=1. > This was due to simplifying the implementation on the client and server. With some additional work, we can satisfy the idempotent guarantees even with any number of in flight requests. The changes on the client be summarized as follows: > > # We increment sequence numbers when batches are drained. > # If for some reason, a batch fails with a retriable error, we know that all future batches would fail with an out of order sequence exception. > # As such, the client should treat some OutOfOrderSequence errors as retriable. In particular, we should maintain the 'last acked sequnece'. If the batch succeeding the last ack'd sequence has an OutOfOrderSequence, that is a fatal error. If a future batch fails with OutOfOrderSequence they should be reenqeued. > # With the changes above, the the producer queues should become priority queues ordered by the sequence numbers. > # The partition is not ready unless the front of the queue has the next expected sequence. > With the changes above, we would get the benefits of multiple inflights in normal cases. When there are failures, we automatically constrain to a single inflight until we get back in sequence. > With multiple inflights, we now have the possibility of getting duplicates for batches other than the last appended batch. In order to return the record metadata (including offset) of the duplicates inside the log, we would require a log scan at the tail to get the metadata at the tail. This can be optimized by caching the metadata for the last 'n' batches. For instance, if the default max.inflight is 5, we could cache the record metadata of the last 5 batches, and fall back to a scan if the duplicate is not within those 5. > * * > The reason to have acks=all is to protect against OutOfOrderSequence exceptions in the case where the leader fails before replication happens. In that case, the next batch sent by the producer would get an OutOfOrderSequence because the new leader would not have the last message. > This may be OK: for applications which really care about avoiding duplicates, they have to handle fatal errors of this sort anyway. In particular, the recommendation is to close the producer in the callback on a fatal error and then check the tail of the log for the last committed message, and then start sending from there. > By making acks=all, this application logic would just be exercised more frequently. -- This message was sent by Atlassian JIRA (v6.4.14#64029)