Hi Chia-Ping, > I have a small question regarding the deprecation of the classic protocol. If > we add deprecation messages to the classic protocol, Streams and Connect > users will also encounter these messages. They may want to understand the > plan for migrating to the new protocol.
This is a very good question. I think that we must differentiate the classic protocol from using the classic protocol with the Consumer. My plan is to deprecate the latter. We cannot deprecate the classic protocol until all components (e.g. Connect) have a story for not using it. We are not there yet. For Streams, I agree that we should not log the deprecation warning as it will be confusing. We can probably have a special condition for Streams in this case. I think that we can discuss those when I open the KIP. Best, David On Thu, Dec 19, 2024 at 4:11 AM Chia-Ping Tsai <chia7...@apache.org> wrote: > > hi David > > Thank you for managing this release. This marks a significant milestone in > Kafka’s history with numerous improvements. While these enhancements may > introduce some inconveniences for users, the revised plan should effectively > bridge the gap. > > I have a small question regarding the deprecation of the classic protocol. If > we add deprecation messages to the classic protocol, Streams and Connect > users will also encounter these messages. They may want to understand the > plan for migrating to the new protocol. > > Thanks again to David. I am eagerly anticipating the release of version 4.0.0. > > Best, > Chia-Ping > > On 2024/12/18 14:08:23 David Jacot wrote: > > Hi all, > > > > As you may know, we've been working hard to graduate KIP-848 from > > preview in Apache Kafka 3.8 to production-ready in Apache Kafka 4.0 — > > exciting news! > > > > KIP-848, the next-generation rebalance protocol, encompasses several > > significant updates in the consumer and group coordinator areas, > > including: > > - A completely new rebalance protocol for the consumer. > > - A redesigned group coordinator. > > - A deep refactoring of the consumer to introduce a background thread > > for handling all I/O operations, instead of performing them on the > > application thread. Note that this refactor is only applied when the > > new rebalance protocol is enabled — an important detail. > > > > Starting with Apache Kafka 4.0: > > - The new group coordinator will be the only option. > > - The new rebalance protocol will be enabled by default on the server side. > > - The consumer will continue to use the old rebalance protocol by > > default for backward compatibility with brokers older than 4.0. > > > > Our initial plan was to make the new rebalance protocol the default in > > the Consumer in Apache Kafka 4.0. However, after much discussion, we > > decided not to proceed with this approach — despite the feature being > > ready. The key concern is that the consumer shipped with Apache Kafka > > 4.0 would not work by default with brokers older than 4.0, which we > > believe would result in a poor user experience. Realistically, most > > deployments won’t upgrade to 4.0 immediately, and we anticipate a > > gradual adoption curve. Moreover, we feel that introducing such a > > change without prior notice (e.g. a deprecation warning) in earlier > > releases isn't ideal. > > > > We also considered implementing an automatic fallback mechanism in the > > consumer. However, the significant differences between the two > > protocols made this unfeasible. For example: > > - Static membership does not function identically in both protocols. > > - Consumer-triggered rebalances are not supported in the new protocol. > > - Default assignors are different. > > - Metrics and logs are different. > > - Some configs are no longer supported. > > - There are small behavioral differences due to the new internal > > architecture of the Consumer. > > > > Given these differences, we believe users must explicitly choose when > > to migrate from the old protocol to the new protocol as already > > planned in the KIP. To aid this process, we are providing a clear > > online upgrade path. > > > > Instead, our revised plan is as follows: > > - We will officially deprecate the old rebalance protocol in a 4.x > > release (likely 4.1) and introduce the appropriate deprecation > > message. > > - We plan to make the new rebalance protocol the default in Apache > > Kafka 5.0, giving the community sufficient time to adopt 4.x brokers > > before the switch. > > - We will drop the possibility of using the old rebalance protocol > > from the Consumer in Apache Kafka 6.0. > > > > I will formalize this revised plan in a KIP to discuss it further with > > the community. > > > > Regards, > > David, on behalf of the KIP-848 team > > > > P.S.: I started this new thread to ensure this update isn’t unnoticed > > in the long KIP-848 discussion thread. > >