Thanks for the KIP Lianet. It's great to see KIP-848 progressing further!


About the question on removing old code in the client. Wondering what we could even remove, given that Connect is still using the "classic" protocol? In the end, Connect calls "new KafkaConsumer()" (for sink connectors) and wants to use "classic" protocol from my understanding. As a matter of fact, it seems (if Connect stays on "classic" for sink connectors), we would need an internal config for Connect on the consumer allowing connect to keep using "classic"?

Or does Connect plan to move to "consumer" protocol for sink connectors?

The fact that Connect uses "WorkerCoordinator" for source connectors instead of "ConsumerCoordinator" seems to be irrelevant as long as sink connectors stay on "classic"?


For testing, I would hope/assume that Connect would cover all testing we would need for the "classic" protocol with sink connectors? I would not "hack" something into system test. And if we really don't feel confident that Connect covers everything we need, we should better keep the old code in the consumer and allow us to enable it with some internal config and keep some test on both protocols? (But maybe we need this anyway -- cf. my question above -- and it would be easy to keep testing both protocols using `KafkaConsumer`).


Might be good to get some input from Connect folks.



MJS1: In Phase 3, you propose to throw a `ConfigException` if `group.protocol` is specified -- are we sure we want to do this? In the end, if we remove `group.protocol` it would become just another unknown consumer config, and we would log a WARN about it (this is code that already exists). Why would we want to move off this behavior? It would be specifically odd, if "consumer" protocol is specified via `group.protocol` and we "force" users to remove the entire entry?


About Phase 2, I agree with Ming-Yen: given that "consumer" is default, and we deprecate `group.protocol`, it seems to be a good idea to log a warning when `group.protocol` is explicitly specified, independent if "classic" or "consumer" is configured (we might of course log a different WARN message for both cases).



-Matthias


On 1/26/26 7:18 PM, Ming-Yen Chung wrote:
Hi Lianet,

I think I didn't express ming_02 clearly.
Let me clarify: In Phase 2, should we also log a deprecation warning when
group.protocol is explicitly set to consumer, suggesting users to remove
the config?

Best,
Ming-Yen

Ming-Yen Chung <[email protected]> 於 2026年1月27日週二 上午10:46寫道:

Hi Lianet,

Thanks for the KIP.

ming_00: The discussion thread link in the KIP appears to be a
placeholder. Could you update it?

ming_01: The KIP mentions "Connect and Schema Registry" as components
using Classic protocol. However, Schema Registry is not part of Apache
Kafka. Should we remove this reference, or clarify that it is a third-party
application?

ming_02: In Phase 2, users who explicitly set group.protocol=consumer
won't receive any warning that this config will be removed in 6.0. Should
we add a deprecation warning whenever group.protocol is explicitly set
(regardless of value)? Or did I misunderstand — does "suggesting upgrade to
the Consumer protocol" actually mean suggesting users to remove the config?

Best,
Ming-Yen

Lianet Magrans <[email protected]> 於 2026年1月27日週二 上午2:02寫道:

Hi Chia, thanks for looking into this!

chia_00: I think it would be great if we could get rid of the classic
consumer code on the AK client 6.0, but agree that we would still need to
verify the broker-side logic. Wonder if we could maybe aim for removing
the
client code, but keep test coverage for the classic protocol with:
- system tests (i.e extending

https://github.com/apache/kafka/blob/trunk/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py
to add tests for produce/consume with client versions < 6.0 selecting the
classic protocol)
- request-level integration test (i.e the ones covering classic protocol
requests like

https://github.com/apache/kafka/blob/trunk/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
,

https://github.com/apache/kafka/blob/trunk/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
,
...)
- Connect tests (while Connect remains using the classic protocol, its
tests would be added coverage)
What do you think? Would that be enough to give us confidence that the
broker support for classic remains stable, while allowing us to clean up
and simplify KafkaConsumer 6.0+ clients?

chia_01: Yes, I think we need to continue fixing bugs on the classic
protocol on the broker to support components like Connect and other client
implementations. That being said, wonder if we should consider something
like fixing only major bugs? (no minor fixes, no new features) I think it
would be helpful to send a strong clear signal to other client
implementations that is the next generation of the rebalance protocol
to adopt on consumer applications (not just another alternative), so they
move and upgrade following a path similar to the java-client. Thoughts?

Thanks!
Lianet

On Thu, Jan 22, 2026 at 5:44 PM Chia-Ping Tsai <[email protected]>
wrote:

hi Lianet

chia_00: Should we keep the classic consumer code in kafka 6.0? I think
the answer is yes. We need it to verify the broker's support for classic
consumer

chia_01: Should we keep fixing bugs in the classic logic for 6.0?

Best,
Chia-Ping

On 2026/01/22 22:04:53 Lianet Magrans wrote:
  Hello,

I would like to start the discussion for KIP-1274. This KIP proposes a
phased plan to drive a smooth evolution of Java client consumer
applications towards the next generation of the rebalance protocol

Here is the KIP:


https://cwiki.apache.org/confluence/display/KAFKA/KIP-1274%3A+Deprecate+and+remove+support+for+Classic+rebalance+protocol+in+KafkaConsumer

Looking forward to hearing from you,

Thanks!
Lianet






Reply via email to