Hi Matthias, About 2: yeah I guess this is a subjective preference. My main concern about keeping the config / handling code beyond 1.2 release is that it will become a non-cleanable tech debt forever, as fewer and fewer users would need to upgrade from 0.10.x and 1.1.x, and eventually we will need to maintain this for nearly no one. On the other hand, I agree that this tech debt is not too large. So if more people feel this is a good tradeoff to pay for not enforcing users from older versions to upgrade twice I'm happen to change my opinion.
A few more minor comments: 4. For the values of "upgrade.from", could we simply to only major.minor? I.e. "0.10.0" than "0.10.0.x" ? Since we never changed compatibility behavior in bug fix releases we would not need to specify a bug-fix version to distinguish ever. 5. Could you also present the encoding format in subscription / assignment metadata bytes in version 2, and in future versions (i.e. which first bytes would be kept moving forward), for readers to better understand the proposal? some snippet like ProduceRequest / ProduceRequest in https://cwiki.apache.org/confluence/display/KAFKA/KIP-98+-+Exactly+Once+Delivery+and+Transactional+Messaging would be very helpful. Guozhang On Fri, Mar 16, 2018 at 2:58 PM, Matthias J. Sax <matth...@confluent.io> wrote: > Thanks for your comments. > > 1. Because the old leader cannot decode the new Subscription it can only > send an empty assignment back. The idea to send empty assignments to all > members is interesting. I will try this out in an PR to see how it behaves. > > 2. I don't see an issue with keeping config `upgrade.from` for future > releases. Personally, I would prefer to not force users to do two > upgrades if they want to go from pre-1.2 to post-1.2 version. Is there a > technical argument why you want to get rid of the config? What > disadvantages do you see keeping `upgrade.from` beyond 1.2 release? > > Keeping the config is just a few lines of code in `StreamsConfig` as > well we a single `if` statement in `StreamsPartitionAssignor` to force a > downgrade (cf > https://github.com/apache/kafka/pull/4636/files#diff- > 392371c29384e33bb09ed342e7696c68R201) > > > 3. I updated the KIP accordingly. > > > -Matthias > > On 3/15/18 3:19 PM, Guozhang Wang wrote: > > Hello Matthias, thanks for the KIP. Here are some comments: > > > > 1. "For all other instances the leader sends a regular Assignment in > > version X back." Does that mean the leader will exclude any member of the > > group whose protocol version that it does not understand? For example, if > > we have A, B, C with A the leader, and B bounced with the newer version. > In > > the first rebalance, A will only consider {A, C} for assignment while > > sending empty assignment to B. And then later when B downgrades will it > > re-assign the tasks to it again? I felt this is unnecessarily increasing > > the num. rebalances and the total latency. Could the leader just sends > > empty assignment to everyone, and since upon receiving the empty > assignment > > each thread will not create / restore any tasks and will not clean up its > > local state (so that the prevCachedTasks are not lost in future > rebalances) > > and re-joins immediately, if users choose to bounce an instance once it > is > > in RUNNING state the total time of rolling upgrades will be reduced. > > > > 2. If we want to allow upgrading from 1.1- versions to any of the future > > versions beyond 1.2, then we'd always need to keep the special handling > > logic for this two rolling-bounce mechanism plus a config that we would > > never be able to deprecate; on the other hand, if the version probing > > procedure is fast, I think the extra operational cost from upgrading from > > 1.1- to a future version, to upgrading from 1.1- to 1.2, and then another > > upgrade from 1.2 to a future version could be small. So depending on the > > experimental result of the upgrade latency, I'd suggest considering the > > trade-off of the extra code/config needed maintaining for the special > > handling. > > > > 3. Testing plan: could you elaborate a bit more on the actual > upgrade-paths > > we should test? For example, I'm thinking the following: > > > > a. 0.10.0 -> 1.2 > > b. 1.1 -> 1.2 > > c. 1.2 -> 1.3 (simulated v4) > > d. 0.10.0 -> 1.3 (simulated v4) > > e. 1.1 -> 1.3 (simulated v4) > > > > Guozhang > > > > > > > > > > On Wed, Mar 14, 2018 at 11:17 PM, Matthias J. Sax <matth...@confluent.io > > > > wrote: > > > >> Hi, > >> > >> I want to propose KIP-268 to allow rebalance metadata version upgrades > >> in Kafka Streams: > >> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP- > >> 268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade > >> > >> Looking forward to your feedback. > >> > >> > >> -Matthias > >> > >> > > > > > > -- -- Guozhang