Hi Kafka Community,
I'd like to start a discussion on KIP-1173: Connect Storage Topics Sharing
Across Clusters
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-1173%3A+Connect+Storage+Topics+Sharing+Across+Clusters>
.
The primary motivation for writing this KIP and proposing this enhancement
came from the operational overhead associated with the creation of
*three storage topics every time when spinning up a new Kafka Connect
Cluster. *While each cluster only requires *three topics*, their cumulative
impact grows significantly as more kafka connect clusters are deployed
not only operationally but also but also from the management, monitoring
and cleaning perspective.
This also makes it very hard to provision the Kafka Connect Clusters on
demand even if operating on the same Kafka Cluster.
But as these topics have very light traffic and are compacted, instead of
provisioning dedicated topics for every cluster, Kafka Connect
clusters can *share
internal topics* across multiple deployments. This brings *immediate
benefits*:
- *Drastically Reduces Topic Proliferation* – Eliminates unnecessary
topic creation.
- *Faster Kafka Connect Cluster Deployment* – No waiting for new topic
provisioning.
- *Large Enterprises with Multiple Teams Using Kafka Connect*
- *Scenario:* In large organisations, multiple teams manage
different *Kafka Connect clusters* for various data pipelines.
- *Benefit:* Instead of waiting for new *internal topics* to be
provisioned each time a new cluster is deployed, teams can
*immediately
start* using pre-existing shared topics, reducing lead time and
improving efficiency.
- *Cloud-Native & Kubernetes-Based Deployments*
- *Scenario:* Many organisations deploy Kafka Connect in
*containerised
environments* (e.g., Kubernetes), where clusters are
frequently *scaled
up/down* or *recreated* dynamically.
- *Benefit:* Since internal topics are already available, new
clusters can *spin up instantly*, without waiting for *topic
provisioning* or *Kafka ACL approvals*.
- How this will help different organisations:
- *Lower Operational Load* – Reduces disk-intensive cleanup operations.
- Broker resource utilization is expected to decrease by
approximately 20%, primarily due to reduced partition count and metadata
overhead. This optimization can enable further cluster downscaling,
contributing directly to lower infrastructure costs (e.g., fewer brokers,
reduced EBS storage footprint, and lower I/O throughput).
- Administrative overhead and monitoring complexity are projected to
reduce by 30%, due to:
- Fewer topics to configure, monitor, and apply
retention/compaction policies to.
- Reduced rebalancing operations during cluster scale-in or
scale-out events.
- *Simplified Management* – Less overhead in monitoring and
maintaining internal topics.
More details on this can be found inside this KIP.
KIP LINK ->
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1173%3A+Connect+Storage+Topics+Sharing+Across+Clusters
Thanks,
Pritam