From dev-return-105746-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Tue Jul 16 00:27:59 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 53E04180595 for ; Tue, 16 Jul 2019 02:27:59 +0200 (CEST) Received: (qmail 81601 invoked by uid 500); 16 Jul 2019 00:27:56 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 81589 invoked by uid 99); 16 Jul 2019 00:27:55 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 16 Jul 2019 00:27:55 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id DA953C2E8C for ; Tue, 16 Jul 2019 00:27:54 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.001 X-Spam-Level: ** X-Spam-Status: No, score=2.001 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, SPF_HELO_NONE=0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, URIBL_CSS=0.1, URIBL_CSS_A=0.1] autolearn=disabled Authentication-Results: spamd4-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=confluent.io Received: from mx1-ec2-va.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id zbx_kYb0XRCZ for ; Tue, 16 Jul 2019 00:27:49 +0000 (UTC) Received-SPF: Pass (mailfrom) identity=mailfrom; client-ip=209.85.167.169; helo=mail-oi1-f169.google.com; envelope-from=jason@confluent.io; receiver= Received: from mail-oi1-f169.google.com (mail-oi1-f169.google.com [209.85.167.169]) by mx1-ec2-va.apache.org (ASF Mail Server at mx1-ec2-va.apache.org) with ESMTPS id 463E0BC77D for ; Tue, 16 Jul 2019 00:27:49 +0000 (UTC) Received: by mail-oi1-f169.google.com with SMTP id 65so14152419oid.13 for ; Mon, 15 Jul 2019 17:27:49 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=confluent.io; s=google; h=mime-version:references:in-reply-to:from:date:message-id:subject:to; bh=/6TldLzfPG2fSl/5k30HR16G9QCXD3knr8Plx0+MviM=; b=HuuKibIXEL5Zk0ZpYvhllzAQQB0ntS/BqmYKkyoqWCOlLgnwFf5J/MQAJkPkYepjt9 57gKi8S7SAtqnxw80FXa5vAQWY6sl1kU4z/yjHAQ2/0/8fKl49POSdQXNs0Qp7yG60RG RD4SS+Mn6qXpbqqum70RogqdLSltSzKNm7LcqlbPcSdzkiwzg3QebAeCL5bD4pvebcdf rXEYyVtGVlLgXxvECn8qIJrggwtiO/IRK1fpr5QhYbebjEkQg1ad1OC2Epkw61Q3Jp1U 1W4vmOSdr4uQtXEnIor1ooHG8Z+46OBpdM6pYLruQe/Eo8CaggihpzBpvZiB0U8UJtWU 9UbA== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:references:in-reply-to:from:date :message-id:subject:to; bh=/6TldLzfPG2fSl/5k30HR16G9QCXD3knr8Plx0+MviM=; b=UhjvLZ2kgbgd4XnbJNk/j2sM5sI27SAD/nhLSRWRjRaF3Y7dBoKf4VMkoQL3uc8C1d k7rUSE+LkT4XAnlh1c543oOl0RUKJh4MrfoKTaxaLFaWjmsmWFpX29rBUDpL0ioG/Xjj /VOTko3r5zjNJIOpcwxumIPU62GEAlwXaFveq7PrZ5ezDN3Q2Bk1CLK+H3Zec8DbSG4s 6O9TutFSU5adJ8Oy4FOgOn1VgjYeoPMxH5rwIPhktxFhm1+8nUmDLI2mrEfzjfd8s/pE H8ZcUgVZzElVxlBBOfM1lO5LmlRvF3P6ep5g+M5pCy4hVwOORaSXka13CG5nPXqw/UkT rbuA== X-Gm-Message-State: APjAAAXAwMJAMH6Mh1htcj88W0IYwOIfJ5jxL3zmHaY+eatsfmRoxCcg 9iGZbboeLHIdwwsSIWlwhu9BWt/buDVvDCcmcvcGvAmQ X-Google-Smtp-Source: APXvYqzxJTL/+xojVvz8AkvJ0Qw7/xILmEglH76CVbIVMFkcbiswe86BZ59ifd8t3OJO+S0mFqWZdGGjE3oM+oEoS6k= X-Received: by 2002:aca:55d7:: with SMTP id j206mr14070852oib.146.1563236867929; Mon, 15 Jul 2019 17:27:47 -0700 (PDT) MIME-Version: 1.0 References: <3c0c23bf-5eb0-44f3-9d05-926ad7cf5284@www.fastmail.com> In-Reply-To: From: Jason Gustafson Date: Mon, 15 Jul 2019 17:27:36 -0700 Message-ID: Subject: Re: [DISCUSS] KIP-447: Producer scalability for exactly once semantics To: dev Content-Type: multipart/alternative; boundary="000000000000369451058dc1726c" --000000000000369451058dc1726c Content-Type: text/plain; charset="UTF-8" Content-Transfer-Encoding: quoted-printable Hi Boyang, Thanks for the updates. A few comments below: 1. The KIP mentions that `transaction.timeout.ms` should be reduced to 10s. I think this makes sense for Kafka Streams which is tied to the consumer group semantics and uses a default 10s session timeout. However, it seems a bit dangerous to make this change for the producer generally. Could we just change it for streams? 2. The new `initTransactions` API takes a `Consumer` instance. I think the idea is to basically put in a backdoor to give the producer access to the group generationId. It's not clear to me how this would work given package restrictions. I wonder if it would be better to just expose the state we need from the consumer. I know we have been reluctant to do this so far because we treat the generationId as an implementation detail. However, I think we might just bite the bullet and expose it rather than coming up with a messy hack. Concepts such as memberIds have already been exposed in the AdminClient, so maybe it is not too bad. Alternatively, we could use an opaque type. For example: // public interface GroupMetadata {} // private interface ConsumerGroupMetadata { final int generationId; final String memberId; } // Consumer API public GroupMetadata groupMetadata(); I am probably leaning toward just exposing the state we need. 3. Given that we are already providing a way to propagate group state from the consumer to the producer, I wonder if we may as well include the memberId and groupInstanceId. This would make the validation we do for TxnOffsetCommit consistent with OffsetCommit. If for no other benefit, at least this may help with debugging. 4. I like the addition of isolation_level to the offset fetch. At the same time, its behavior is a bit inconsistent with how it is used in the consumer generally. There is no reason for the group coordinator to ever expose aborted data, so this is mostly about awaiting pending offset commits, not reading uncommitted data. Perhaps instead of calling this "isolation level," it should be more like "await_pending_transaction" or something like that? Also, just to be clear, the consumer would treat this as an optional field, right? So if the broker does not support the latest OffsetFetch API, it would silently revert to reading the old data. Basically it would be up to the streams version probing logic to ensure that the expectation on this API fits with the usage of `transctional.id`. Thanks, Jason On Mon, Jul 8, 2019 at 3:19 PM Boyang Chen wrote: > Hey Guozhang, > > I will correct my statement from last email. I don't think the > read_committed (3.a) is necessary to be added to the OffsetFetch request, > as if we are using EOS application, the underlying consumers within the > group should always back off when there is pending offsets. > > Let me know if you think this is correct. > > On Tue, Jul 2, 2019 at 3:21 PM Boyang Chen > wrote: > > > Thank you Guozhang for the questions, inline answers are below. > > > > On Tue, Jul 2, 2019 at 3:14 PM Boyang Chen > > wrote: > > > >> Hey all, > >> > >> I have done a fundamental polish of KIP-447 > >> < > https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+sca= lability+for+exactly+once+semantics> > and > >> written a design doc > >> < > https://docs.google.com/document/d/1LhzHGeX7_Lay4xvrEXxfciuDWATjpUXQhrEIk= ph9qRE/edit#> > depicting > >> internal changes. We stripped off many implementation details from the > KIP, > >> and simplified the public changes by a lot. For reviewers, it is highl= y > >> recommended to fully understand EOS design in KIP-98 and read its > >> corresponding design doc > >> < > https://docs.google.com/document/d/11Jqy_GjUGtdXJK94XGsEIK7CP1SnQGdp2eF0w= Sw9ra8/edit> > if > >> you haven't done so already. > >> > >> Let me know if you found anything confusing around the KIP or the > design. > >> Would be happy to discuss in depth. > >> > >> Best, > >> Boyang > >> > >> > >> > >> > >> > >> On Wed, Jun 26, 2019 at 11:00 AM Guozhang Wang > >> wrote: > >> > >>> 2. The reason we did not expose generation.id from KafkaConsumer > public > >>> APIs directly is to abstract this notion from users (since it is an > >>> implementation detail of the rebalance protocol itself, e.g. if user > >>> calls > >>> consumer.assign() they do not need to invoke ConsumerCoordinator and = no > >>> need to be aware of generation.id at all). > >>> > >>> On the other hand, with the current proposal the txn.coordiantor did > not > >>> know about the latest generation from the source-of-truth > >>> group.coordinator; instead, it will only bump up the generation from > the > >>> producer's InitProducerIdRequest only. > >>> > >>> The key here is that GroupCoordinator, when handling > >>> `InitProducerIdRequest > >>> > >> In the new design, we just pass the entire consumer instance into the > > producer through > > #initTransaction, so no public API will be created. > > > >> 3. I agree that if we rely on the group coordinator to block on > returning > >>> offset-fetch-response if read-committed is enabled, then we do not ne= ed > >>> to > >>> store partition assignment on txn coordinator and therefore it's bett= er > >>> to > >>> still decouple them. For that case we still need to update the KIP wi= ki > >>> page that includes: > >>> > >>> 3.a. Augment OffsetFetchRequest with the ISOLATION_LEVEL as well. > >>> 3.b. Add new error code in OffsetFetchResponse to let client backoff > and > >>> retry if there are pending txns including the interested partitions. > >>> 3.c. Also in the worst case we would let the client be blocked for th= e > >>> txn.timeout period, and for that rationale we may need to consider > >>> reducing > >>> our default txn.timeout value as well. > >>> > >>> Addressed 3.b and 3.c, will do 3.a. > > > >> 4. According to Colin it seems we do not need to create another KIP an= d > we > >>> can just complete it as part of KIP-117 / KAFKA-5214; and we need to = do > >>> some cleanup to have BrokerApiVersion exposed from AdminClient (@Coli= n > >>> please let use know if you have any concerns exposing it). > >>> > >> I think we no longer need to rely on api version for initialization, > > since we will be using the upgrade.from config anyway. > > > >> > >>> Guozhang > >>> > >>> > >>> On Tue, Jun 25, 2019 at 6:43 PM Jason Gustafson > >>> wrote: > >>> > >>> > For reference, we have BrokerApiVersionCommand already as a public > >>> > interface. We have a bit of tech debt at the moment because it uses= a > >>> > custom AdminClient. It would be nice to clean that up. In general, = I > >>> think > >>> > it is reasonable to expose from AdminClient. It can be used by > >>> management > >>> > tools to inspect running Kafka versions for example. > >>> > > >>> > -Jason > >>> > > >>> > On Tue, Jun 25, 2019 at 4:37 PM Boyang Chen < > >>> reluctanthero104@gmail.com> > >>> > wrote: > >>> > > >>> > > Thank you for the context Colin. The groupId was indeed a > copy-paste > >>> > error. > >>> > > Our use case here for 447 is (Quoted from Guozhang): > >>> > > ''' > >>> > > I think if we can do something else to > >>> > > avoid this config though, for example we can use the embedded > >>> AdminClient > >>> > > to send the APIVersion request upon starting up, and based on the > >>> > returned > >>> > > value decides whether to go to the old code path or the new > behavior. > >>> > > ''' > >>> > > The benefit we get is to avoid adding a new configuration to make= a > >>> > > decision simply base on broker version. If you have concerns with > >>> > exposing > >>> > > ApiVersion for client, we could > >>> > > try to think of alternative solutions too. > >>> > > > >>> > > Boyang > >>> > > > >>> > > > >>> > > > >>> > > On Tue, Jun 25, 2019 at 4:20 PM Colin McCabe > >>> wrote: > >>> > > > >>> > > > kafka.api.ApiVersion is an internal class, not suitable to > exposing > >>> > > > through AdminClient. That class is not even accessible without > >>> having > >>> > > the > >>> > > > broker jars on your CLASSPATH. > >>> > > > > >>> > > > Another question is, what is the groupId parameter doing in the > >>> call? > >>> > > The > >>> > > > API versions are the same no matter what consumer group we use, > >>> right? > >>> > > > Perhaps this was a copy and paste error? > >>> > > > > >>> > > > This is not the first time we have discussed having a method in > >>> > > > AdminClient to retrieve API version information. In fact, the > >>> original > >>> > > KIP > >>> > > > which created KafkaAdminClient specified an API for fetching > >>> version > >>> > > > information. It was called apiVersions and it is still there o= n > >>> the > >>> > > wiki. > >>> > > > See > >>> > > > > >>> > > > >>> > > >>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-117%3A+Add+a+public= +AdminClient+API+for+Kafka+admin+operations > >>> > > > > >>> > > > However, this API wasn't ready in time for 0.11.0 so we shipped > >>> without > >>> > > > it. There was a JIRA to implement it for later versions, > >>> > > > https://issues.apache.org/jira/browse/KAFKA-5214 , as well as a > >>> PR, > >>> > > > https://github.com/apache/kafka/pull/3012 . However, we starte= d > >>> to > >>> > > > rethink whether this AdminClient function was even necessary. > >>> Most of > >>> > > the > >>> > > > use-cases we could think of seemed like horrible hacks. So it > has > >>> > never > >>> > > > really been implemented (yet?). > >>> > > > > >>> > > > best, > >>> > > > Colin > >>> > > > > >>> > > > > >>> > > > On Tue, Jun 25, 2019, at 15:46, Boyang Chen wrote: > >>> > > > > Actually, after a second thought, I think it actually makes > >>> sense to > >>> > > > > support auto upgrade through admin client to help use get api > >>> version > >>> > > > > from > >>> > > > > broker. > >>> > > > > A draft KIP is here: > >>> > > > > > >>> > > > > >>> > > > >>> > > >>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-483%3A++Add+Broker+= Version+API+in+Admin+Client > >>> > > > > > >>> > > > > Boyang > >>> > > > > > >>> > > > > On Tue, Jun 25, 2019 at 2:57 PM Boyang Chen < > >>> > > reluctanthero104@gmail.com> > >>> > > > > wrote: > >>> > > > > > >>> > > > > > Thank you Guozhang, some of my understandings are inline > below. > >>> > > > > > > >>> > > > > > On Tue, Jun 25, 2019 at 11:05 AM Jason Gustafson < > >>> > jason@confluent.io > >>> > > > > >>> > > > > > wrote: > >>> > > > > > > >>> > > > > >> > > >>> > > > > >> > I think co-locating does have some merits here, i.e. > >>> letting the > >>> > > > > >> > ConsumerCoordinator which has the source-of-truth of > >>> assignment > >>> > to > >>> > > > act > >>> > > > > >> as > >>> > > > > >> > the TxnCoordinator as well; but I agree there's also som= e > >>> cons > >>> > of > >>> > > > > >> coupling > >>> > > > > >> > them together. I'm still a bit inclining towards > colocation > >>> but > >>> > if > >>> > > > there > >>> > > > > >> > are good rationales not to do so I can be convinced as > well. > >>> > > > > >> > >>> > > > > >> > >>> > > > > >> The good rationale is that we have no mechanism to colocat= e > >>> > > > partitions ;). > >>> > > > > >> Are you suggesting we store the group and transaction stat= e > >>> in the > >>> > > > same > >>> > > > > >> log? Can you be more concrete about the benefit? > >>> > > > > >> > >>> > > > > >> -Jason > >>> > > > > >> > >>> > > > > >> On Tue, Jun 25, 2019 at 10:51 AM Guozhang Wang < > >>> > wangguoz@gmail.com> > >>> > > > > >> wrote: > >>> > > > > >> > >>> > > > > >> > Hi Boyang, > >>> > > > > >> > > >>> > > > > >> > 1. One advantage of retry against on-hold is that it wil= l > >>> not > >>> > > > tie-up a > >>> > > > > >> > handler thread (of course the latter could do the same b= ut > >>> that > >>> > > > involves > >>> > > > > >> > using a purgatory which is more complicated), and also i= t > is > >>> > less > >>> > > > > >> likely to > >>> > > > > >> > violate request timeout. So I think there are some > >>> rationales to > >>> > > > prefer > >>> > > > > >> > retries. > >>> > > > > >> > > >>> > > > > >> > >>> > > > > > That sounds fair to me, also we are avoiding usage of > another > >>> > > > purgatory > >>> > > > > > instance. Usually for one back-off > >>> > > > > > we are only delaying 50ms during startup which is trivial > cost. > >>> > This > >>> > > > > > behavior shouldn't be changed. > >>> > > > > > > >>> > > > > > > 2. Regarding "ConsumerRebalanceListener": both > >>> > > > ConsumerRebalanceListener > >>> > > > > >> > and PartitionAssignors are user-customizable modules, an= d > >>> only > >>> > > > > >> difference > >>> > > > > >> > is that the former is specified via code and the latter = is > >>> > > > specified via > >>> > > > > >> > config. > >>> > > > > >> > > >>> > > > > >> > Regarding Jason's proposal of ConsumerAssignment, one > thing > >>> to > >>> > > note > >>> > > > > >> though > >>> > > > > >> > with KIP-429 the onPartitionAssigned may not be called i= f > >>> the > >>> > > > assignment > >>> > > > > >> > does not change, whereas onAssignment would always be > >>> called at > >>> > > the > >>> > > > end > >>> > > > > >> of > >>> > > > > >> > sync-group response. My proposed semantics is that > >>> > > > > >> > `RebalanceListener#onPartitionsXXX` are used for > >>> notifications > >>> > to > >>> > > > user, > >>> > > > > >> and > >>> > > > > >> > hence if there's no changes these will not be called, > >>> whereas > >>> > > > > >> > `PartitionAssignor` is used for assignor logic, whose > >>> callback > >>> > > would > >>> > > > > >> always > >>> > > > > >> > be called no matter if the partitions have changed or no= t. > >>> > > > > >> > >>> > > > > >> I think a third option is to gracefully expose generation = id > >>> as > >>> > part > >>> > > > of > >>> > > > > > consumer API, so that we don't need to > >>> > > > > > bother overloading various callbacks. Of course, this build= s > >>> upon > >>> > the > >>> > > > > > assumption that topic partitions > >>> > > > > > will not be included in new initTransaction API. > >>> > > > > > > >>> > > > > > > 3. I feel it is a bit awkward to let the TxnCoordinator > >>> keeping > >>> > > > partition > >>> > > > > >> > assignments since it is sort of taking over the job of t= he > >>> > > > > >> > ConsumerCoordinator, and may likely cause a split-brain > >>> problem > >>> > as > >>> > > > two > >>> > > > > >> > coordinators keep a copy of this assignment which may be > >>> > > different. > >>> > > > > >> > > >>> > > > > >> > I think co-locating does have some merits here, i.e. > >>> letting the > >>> > > > > >> > ConsumerCoordinator which has the source-of-truth of > >>> assignment > >>> > to > >>> > > > act > >>> > > > > >> as > >>> > > > > >> > the TxnCoordinator as well; but I agree there's also som= e > >>> cons > >>> > of > >>> > > > > >> coupling > >>> > > > > >> > them together. I'm still a bit inclining towards > colocation > >>> but > >>> > if > >>> > > > there > >>> > > > > >> > are good rationales not to do so I can be convinced as > well. > >>> > > > > >> > > >>> > > > > >> > >>> > > > > > The purpose of co-location is to let txn coordinator see th= e > >>> group > >>> > > > > > assignment. This priority is weakened > >>> > > > > > when we already have defense on the consumer offset fetch, > so I > >>> > guess > >>> > > > it's > >>> > > > > > not super important anymore. > >>> > > > > > > >>> > > > > > > >>> > > > > >> > 4. I guess I'm preferring the philosophy of "only add > >>> configs if > >>> > > > > >> there's no > >>> > > > > >> > other ways", since more and more configs would make it > less > >>> and > >>> > > less > >>> > > > > >> > intuitive out of the box to use. > >>> > > > > >> > > >>> > > > > >> > I think it's a valid point that checks upon starting up > >>> does not > >>> > > > cope > >>> > > > > >> with > >>> > > > > >> > brokers downgrading but even with a config, but it is > still > >>> hard > >>> > > for > >>> > > > > >> users > >>> > > > > >> > to determine when they can be ensured the broker would > never > >>> > > > downgrade > >>> > > > > >> > anymore and hence can safely switch the config. So my > >>> feeling is > >>> > > > that > >>> > > > > >> this > >>> > > > > >> > config would not be helping too much still. If we want t= o > >>> be at > >>> > > the > >>> > > > > >> safer > >>> > > > > >> > side, then I'd suggest we modify the Coordinator -> > >>> > NetworkClient > >>> > > > > >> hierarchy > >>> > > > > >> > to allow the NetworkClient being able to pass the > APIVersion > >>> > > > metadata to > >>> > > > > >> > Coordinator, so that Coordinator can rely on that logic = to > >>> > change > >>> > > > its > >>> > > > > >> > behavior dynamically. > >>> > > > > >> > >>> > > > > > The stream thread init could not be supported by a client > >>> > coordinator > >>> > > > > > behavior change on the fly, > >>> > > > > > we are only losing possibilities after we initialized. (mai= n > >>> thread > >>> > > > gets > >>> > > > > > exit and no thread has global picture anymore) > >>> > > > > > If we do want to support auto version detection, admin clie= nt > >>> > request > >>> > > > in > >>> > > > > > this sense shall be easier. > >>> > > > > > > >>> > > > > > > >>> > > > > >> > > >>> > > > > >> > 5. I do not have a concrete idea about how the impact on > >>> Connect > >>> > > > would > >>> > > > > >> > make, maybe Randall or Konstantine can help here? > >>> > > > > >> > > >>> > > > > >> > >>> > > > > > Sounds good, let's see their thoughts. > >>> > > > > > > >>> > > > > > > >>> > > > > >> > Guozhang > >>> > > > > >> > > >>> > > > > >> > On Mon, Jun 24, 2019 at 10:26 PM Boyang Chen < > >>> > > > > >> reluctanthero104@gmail.com> > >>> > > > > >> > wrote: > >>> > > > > >> > > >>> > > > > >> > > Hey Jason, > >>> > > > > >> > > > >>> > > > > >> > > thank you for the proposal here. Some of my thoughts > >>> below. > >>> > > > > >> > > > >>> > > > > >> > > On Mon, Jun 24, 2019 at 8:58 PM Jason Gustafson < > >>> > > > jason@confluent.io> > >>> > > > > >> > > wrote: > >>> > > > > >> > > > >>> > > > > >> > > > Hi Boyang, > >>> > > > > >> > > > > >>> > > > > >> > > > Thanks for picking this up! Still reading through th= e > >>> > updates, > >>> > > > but > >>> > > > > >> here > >>> > > > > >> > > are > >>> > > > > >> > > > a couple initial comments on the APIs: > >>> > > > > >> > > > > >>> > > > > >> > > > 1. The `TxnProducerIdentity` class is a bit awkward.= I > >>> think > >>> > > we > >>> > > > are > >>> > > > > >> > > trying > >>> > > > > >> > > > to encapsulate state from the current group > assignment. > >>> > Maybe > >>> > > > > >> something > >>> > > > > >> > > > like `ConsumerAssignment` would be clearer? If we ma= ke > >>> the > >>> > > usage > >>> > > > > >> > > consistent > >>> > > > > >> > > > across the consumer and producer, then we can avoid > >>> exposing > >>> > > > > >> internal > >>> > > > > >> > > state > >>> > > > > >> > > > like the generationId. > >>> > > > > >> > > > > >>> > > > > >> > > > For example: > >>> > > > > >> > > > > >>> > > > > >> > > > // Public API > >>> > > > > >> > > > interface ConsumerAssignment { > >>> > > > > >> > > > Set partittions(); > >>> > > > > >> > > > } > >>> > > > > >> > > > > >>> > > > > >> > > > // Not a public API > >>> > > > > >> > > > class InternalConsumerAssignment implements > >>> > > ConsumerAssignment { > >>> > > > > >> > > > Set partittions; > >>> > > > > >> > > > int generationId; > >>> > > > > >> > > > } > >>> > > > > >> > > > > >>> > > > > >> > > > Then we can change the rebalance listener to somethi= ng > >>> like > >>> > > > this: > >>> > > > > >> > > > onPartitionsAssigned(ConsumerAssignment assignment) > >>> > > > > >> > > > > >>> > > > > >> > > > And on the producer: > >>> > > > > >> > > > void initTransactions(String groupId, > ConsumerAssignment > >>> > > > > >> assignment); > >>> > > > > >> > > > > >>> > > > > >> > > > 2. Another bit of awkwardness is the fact that we ha= ve > >>> to > >>> > pass > >>> > > > the > >>> > > > > >> > > groupId > >>> > > > > >> > > > through both initTransactions() and > >>> > > sendOffsetsToTransaction(). > >>> > > > We > >>> > > > > >> > could > >>> > > > > >> > > > consider a config instead. Maybe something like ` > >>> > > > > >> > transactional.group.id > >>> > > > > >> > > `? > >>> > > > > >> > > > Then we could simplify the producer APIs, potentiall= y > >>> even > >>> > > > > >> deprecating > >>> > > > > >> > > the > >>> > > > > >> > > > current sendOffsetsToTransaction. In fact, for this > new > >>> > usage, > >>> > > > the ` > >>> > > > > >> > > > transational.id` config is not needed. It would be > >>> nice if > >>> > we > >>> > > > don't > >>> > > > > >> > have > >>> > > > > >> > > > to > >>> > > > > >> > > > provide it. > >>> > > > > >> > > > > >>> > > > > >> > > > I like the idea of combining 1 and 2. We could > >>> definitely > >>> > pass > >>> > > > in a > >>> > > > > >> > > group.id config > >>> > > > > >> > > so that we could avoid exposing that information in a > >>> public > >>> > > API. > >>> > > > The > >>> > > > > >> > > question I have > >>> > > > > >> > > is that whether we should name the interface > >>> `GroupAssignment` > >>> > > > > >> instead, > >>> > > > > >> > so > >>> > > > > >> > > that Connect later > >>> > > > > >> > > could also extend on the same interface, just to echo > >>> > Guozhang's > >>> > > > point > >>> > > > > >> > > here, Also the base interface > >>> > > > > >> > > is better to be defined empty for easy extension, or > >>> define an > >>> > > > > >> abstract > >>> > > > > >> > > type called `Resource` to be shareable > >>> > > > > >> > > later IMHO. > >>> > > > > >> > > > >>> > > > > >> > > > >>> > > > > >> > > > By the way, I'm a bit confused about discussion abov= e > >>> about > >>> > > > > >> colocating > >>> > > > > >> > > the > >>> > > > > >> > > > txn and group coordinators. That is not actually > >>> necessary, > >>> > is > >>> > > > it? > >>> > > > > >> > > > > >>> > > > > >> > > > Yes, this is not a requirement for this KIP, because > it > >>> is > >>> > > > > >> inherently > >>> > > > > >> > > impossible to > >>> > > > > >> > > achieve co-locating topic partition of transaction lo= g > >>> and > >>> > > > consumed > >>> > > > > >> > offset > >>> > > > > >> > > topics. > >>> > > > > >> > > > >>> > > > > >> > > > >>> > > > > >> > > > Thanks, > >>> > > > > >> > > > Jason > >>> > > > > >> > > > > >>> > > > > >> > > On Mon, Jun 24, 2019 at 10:07 AM Boyang Chen < > >>> > > > > >> reluctanthero104@gmail.com > >>> > > > > >> > > > >>> > > > > >> > > > wrote: > >>> > > > > >> > > > > >>> > > > > >> > > > > Thank you Ismael for the suggestion. We will attem= pt > >>> to > >>> > > > address > >>> > > > > >> it by > >>> > > > > >> > > > > giving more details to rejected alternative sectio= n. > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > Thank you for the comment Guozhang! Answers are > inline > >>> > > below. > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > On Sun, Jun 23, 2019 at 6:33 PM Guozhang Wang < > >>> > > > wangguoz@gmail.com > >>> > > > > >> > > >>> > > > > >> > > > wrote: > >>> > > > > >> > > > > > >>> > > > > >> > > > > > Hello Boyang, > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > Thanks for the KIP, I have some comments below: > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > 1. "Once transactions are complete, the call wil= l > >>> > return." > >>> > > > This > >>> > > > > >> > seems > >>> > > > > >> > > > > > different from the existing behavior, in which w= e > >>> would > >>> > > > return a > >>> > > > > >> > > > > retriable > >>> > > > > >> > > > > > CONCURRENT_TRANSACTIONS and let the client to > >>> retry, is > >>> > > this > >>> > > > > >> > > > intentional? > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > I don=E2=80=99t think it is intentional, and I wil= l defer > this > >>> > > > question to > >>> > > > > >> > > Jason > >>> > > > > >> > > > > when he got time to answer since from what I > >>> understood > >>> > > retry > >>> > > > and > >>> > > > > >> on > >>> > > > > >> > > hold > >>> > > > > >> > > > > seem both valid approaches. > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > > 2. "an overload to onPartitionsAssigned in the > >>> > consumer's > >>> > > > > >> rebalance > >>> > > > > >> > > > > > listener interface": as part of KIP-341 we've > >>> already > >>> > add > >>> > > > this > >>> > > > > >> > > > > information > >>> > > > > >> > > > > > to the onAssignment callback. Would this be > >>> sufficient? > >>> > Or > >>> > > > more > >>> > > > > >> > > > generally > >>> > > > > >> > > > > > speaking, which information have to be passed > >>> around in > >>> > > > > >> rebalance > >>> > > > > >> > > > > callback > >>> > > > > >> > > > > > while others can be passed around in > >>> PartitionAssignor > >>> > > > > >> callback? In > >>> > > > > >> > > > > Streams > >>> > > > > >> > > > > > for example both callbacks are used but most > >>> critical > >>> > > > > >> information > >>> > > > > >> > is > >>> > > > > >> > > > > passed > >>> > > > > >> > > > > > via onAssignment. > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > We still need to extend ConsumerRebalanceListener > >>> because > >>> > > > it=E2=80=99s the > >>> > > > > >> > > > > interface we could have public access to. The > >>> > #onAssignment > >>> > > > call > >>> > > > > >> is > >>> > > > > >> > > > defined > >>> > > > > >> > > > > on PartitionAssignor level which is not easy to wo= rk > >>> with > >>> > > > external > >>> > > > > >> > > > > producers. > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > > 3. "We propose to use a separate record type in > >>> order to > >>> > > > store > >>> > > > > >> the > >>> > > > > >> > > > group > >>> > > > > >> > > > > > assignment.": hmm, I thought with the third type= d > >>> > > > > >> FindCoordinator, > >>> > > > > >> > > the > >>> > > > > >> > > > > same > >>> > > > > >> > > > > > broker that act as the consumer coordinator wou= ld > >>> > always > >>> > > be > >>> > > > > >> > selected > >>> > > > > >> > > > as > >>> > > > > >> > > > > > the txn coordinator, in which case it can access > its > >>> > local > >>> > > > cache > >>> > > > > >> > > > > metadata / > >>> > > > > >> > > > > > offset topic to get this information already? We > >>> just > >>> > need > >>> > > > to > >>> > > > > >> think > >>> > > > > >> > > > about > >>> > > > > >> > > > > > how to make these two modules directly exchange > >>> > > information > >>> > > > > >> without > >>> > > > > >> > > > > messing > >>> > > > > >> > > > > > up the code hierarchy. > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > These two coordinators will be on the same broker > only > >>> > when > >>> > > > > >> number of > >>> > > > > >> > > > > partitions for transaction state topic and consume= r > >>> offset > >>> > > > topic > >>> > > > > >> are > >>> > > > > >> > > the > >>> > > > > >> > > > > same. This normally holds true, but I'm afraid > >>> > > > > >> > > > > we couldn't make this assumption? > >>> > > > > >> > > > > > >>> > > > > >> > > > > 4. The config of "CONSUMER_GROUP_AWARE_TRANSACTION= ": > >>> it > >>> > > seems > >>> > > > the > >>> > > > > >> > goal > >>> > > > > >> > > of > >>> > > > > >> > > > > > this config is just to avoid old-versioned broke= r > >>> to not > >>> > > be > >>> > > > > >> able to > >>> > > > > >> > > > > > recognize newer versioned client. I think if we > can > >>> do > >>> > > > something > >>> > > > > >> > else > >>> > > > > >> > > > to > >>> > > > > >> > > > > > avoid this config though, for example we can use > the > >>> > > > embedded > >>> > > > > >> > > > AdminClient > >>> > > > > >> > > > > > to send the APIVersion request upon starting up, > and > >>> > based > >>> > > > on > >>> > > > > >> the > >>> > > > > >> > > > > returned > >>> > > > > >> > > > > > value decides whether to go to the old code path > or > >>> the > >>> > > new > >>> > > > > >> > behavior. > >>> > > > > >> > > > > > Admittedly asking a random broker about APIVersi= on > >>> does > >>> > > not > >>> > > > > >> > guarantee > >>> > > > > >> > > > the > >>> > > > > >> > > > > > whole cluster's versions, but what we can do is = to > >>> first > >>> > > 1) > >>> > > > find > >>> > > > > >> > the > >>> > > > > >> > > > > > coordinator (and if the random broker does not > even > >>> > > > recognize > >>> > > > > >> the > >>> > > > > >> > new > >>> > > > > >> > > > > > discover type, fall back to old path directly), > and > >>> then > >>> > > 2) > >>> > > > ask > >>> > > > > >> the > >>> > > > > >> > > > > > discovered coordinator about its supported > >>> APIVersion. > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > The caveat here is that we have to make sure both > the > >>> > group > >>> > > > > >> > coordinator > >>> > > > > >> > > > and > >>> > > > > >> > > > > transaction coordinator are on the latest version > >>> during > >>> > > init > >>> > > > > >> stage. > >>> > > > > >> > > This > >>> > > > > >> > > > > is potentially doable as we only need a consumer > >>> group.id > >>> > > > > >> > > > > to check that. In the meantime, a hard-coded confi= g > is > >>> > > still a > >>> > > > > >> > > favorable > >>> > > > > >> > > > > backup in case the server has downgraded, so you > will > >>> want > >>> > > to > >>> > > > use > >>> > > > > >> a > >>> > > > > >> > new > >>> > > > > >> > > > > version client without `consumer group` > transactional > >>> > > support. > >>> > > > > >> > > > > > >>> > > > > >> > > > > 5. This is a meta question: have you considered ho= w > >>> this > >>> > can > >>> > > > be > >>> > > > > >> > applied > >>> > > > > >> > > > to > >>> > > > > >> > > > > > Kafka Connect as well? For example, for source > >>> > connectors, > >>> > > > the > >>> > > > > >> > > > assignment > >>> > > > > >> > > > > > is not by "partitions", but by some other sort o= f > >>> > > > "resources" > >>> > > > > >> based > >>> > > > > >> > > on > >>> > > > > >> > > > > the > >>> > > > > >> > > > > > source systems, how KIP-447 would affect Kafka > >>> > Connectors > >>> > > > that > >>> > > > > >> > > > > implemented > >>> > > > > >> > > > > > EOS as well? > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > No, it's not currently included in the scope. Coul= d > >>> you > >>> > > point > >>> > > > me > >>> > > > > >> to a > >>> > > > > >> > > > > sample source connector who uses EOS? Could always > >>> > > piggy-back > >>> > > > into > >>> > > > > >> > the > >>> > > > > >> > > > > TxnProducerIdentity struct with more information > such > >>> as > >>> > > > tasks. If > >>> > > > > >> > > > > this is something to support in near term, an > abstract > >>> > type > >>> > > > called > >>> > > > > >> > > > > "Resource" could be provided and let topic partiti= on > >>> and > >>> > > > connect > >>> > > > > >> task > >>> > > > > >> > > > > implement it. > >>> > > > > >> > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > Guozhang > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > On Sat, Jun 22, 2019 at 8:40 PM Ismael Juma < > >>> > > > ismael@juma.me.uk> > >>> > > > > >> > > wrote: > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > > Hi Boyang, > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > Thanks for the KIP. It's good that we listed a > >>> number > >>> > of > >>> > > > > >> rejected > >>> > > > > >> > > > > > > alternatives. It would be helpful to have an > >>> > explanation > >>> > > > of > >>> > > > > >> why > >>> > > > > >> > > they > >>> > > > > >> > > > > were > >>> > > > > >> > > > > > > rejected. > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > Ismael > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > On Sat, Jun 22, 2019 at 8:31 PM Boyang Chen < > >>> > > > > >> bchen11@outlook.com > >>> > > > > >> > > > >>> > > > > >> > > > > wrote: > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > > Hey all, > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > I would like to start a discussion for > KIP-447: > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > >>> > > > > >> > > > >>> > > > > >> > > >>> > > > > >> > >>> > > > > >>> > > > >>> > > >>> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+sca= lability+for+exactly+once+semantics > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > this is a work originated by Jason Gustafson > >>> and we > >>> > > > would > >>> > > > > >> like > >>> > > > > >> > to > >>> > > > > >> > > > > > proceed > >>> > > > > >> > > > > > > > into discussion stage. > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > Let me know your thoughts, thanks! > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > Boyang > >>> > > > > >> > > > > > > > > >>> > > > > >> > > > > > > > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > -- > >>> > > > > >> > > > > > -- Guozhang > >>> > > > > >> > > > > > > >>> > > > > >> > > > > > >>> > > > > >> > > > > >>> > > > > >> > > > >>> > > > > >> > > >>> > > > > >> > > >>> > > > > >> > -- > >>> > > > > >> > -- Guozhang > >>> > > > > >> > > >>> > > > > >> > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > >>> > >>> > >>> -- > >>> -- Guozhang > >>> > >> > --000000000000369451058dc1726c--