KAFKA-9216: Enforce internal config topic settings for Connect workers during startup (#8270)
Currently, Kafka Connect creates its config backing topic with a fire and forget approach.
This is fine unless someone has manually created that topic already with the wrong partition count.
In such a case Kafka Connect may run for some time. Especially if it's in standalone mode and once switched to distributed mode it will almost certainly fail.
This commits adds a check when the KafkaConfigBackingStore is starting.
This check will throw a ConfigException if there is more than one partition in the backing store.
This exception is then caught upstream and logged by either:
- DistributedHerder#run
- ConnectStandalone#main
A unit tests was added in KafkaConfigBackingStoreTest to verify the behaviour.
Author: Evelyn Bayes <evelyn@confluent.io>
Co-authored-by: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
@ -265,6 +265,16 @@ public class KafkaConfigBackingStore implements ConfigBackingStore {
@@ -265,6 +265,16 @@ public class KafkaConfigBackingStore implements ConfigBackingStore {
// Before startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that
// updates can continue to occur in the background
configLog.start();
intpartitionCount=configLog.partitionCount();
if(partitionCount>1){
Stringmsg=String.format("Topic '%s' supplied via the '%s' property is required "
+"to have a single partition in order to guarantee consistency of "
+"connector configurations, but found %d partitions.",