Hi chris, thank you very much for the review and the feedback, 1. On read amplification part, the bulk read where the whole topic is read will happen on when the cluster is restarted and apart from that it will only be reading the additional records that also by discarding the records of other clusters by just looking at the topic. In case of status topics also, in the long term the growth will be restricted as the topics are compacted. Please correct me if I am wrong or if I am missing something here. We did the perf by rebalancing the cluster and restarting the tasks frequently and we did not notice any observable effect. 2. For the transaction in case of exactly once source connectors, I agree with the view that we can have the dedicated topic per connector for the source connector as that is not a very frequently used use case. Please suggest if there is anything apart from this that we can do to avoid this scenario. 3. For the rebalancing logic, yes they could just ignore the records belonging to different clusters by just looking at the records key. 4. For the GroupId structure, yes I was thinking of introducing a validation logic which will validate the groupId for not containing the "special character - in this case '.'", or we can use JSON also as you rightly pointed out.
Thanks, Pritam On Sat, May 3, 2025 at 9:57 PM Chris Egerton <fearthecel...@gmail.com> wrote: > Hi Pritam, > > I think Greg's concerns about read amplification are still unaddressed, no? > I'd guess that the config topic is unlikely to experience an increase in > load large enough to cause problems, but what about the status topic > (especially if there's, e.g. a K8s operator constantly restarting failed > tasks on the cluster), and the offsets topic? > > There's also the concern about hanging transactions on offsets topics when > running exactly-once source connectors. This can technically be addressed > by creating a dedicated topic per connector, or maybe using a shared topic > for a subset of connectors, but then wouldn't that defeat the purpose of > the KIP? I guess sharing 2 out of 3 topics would still be nice in terms of > reducing partition bloat, but it wouldn't fully address the problems with > organizational constraints on topic creation. > > I'm also unsure how this would work with our rebalancing logic, where > workers report the latest offset in the config topic that they've read up > to in order to ensure that they have a consistent view of the config topic. > I guess they could just ignore offsets for records that correspond to > different clusters? > > Finally, I think we may want to use a more structured format for record > keys. IIRC there's essentially no limits on what characters can be used for > group IDs (please correct me if I'm wrong), so we can't use any kind of > delimiter that won't be included in the group ID. Maybe we can just use > JSON? > > Cheers, > > Chris > > On Sat, May 3, 2025, 12:04 pritam kumar <kumarpritamm...@gmail.com> wrote: > > > Hi Greg, > > Thank you for your review, and apologies for the delayed response—I was > out > > of office. > > *"Broker resource utilization is expected to decrease by approximately > 20%, > > primarily due to reduced partition count and metadata overhead."* > > To clarify, this estimate compares two scenarios: > > 1. Clusters with dedicated internal topics vs. > > 2. A single shared topic for all clusters on the same Kafka cluster. > > The 20% reduction refers to the relative improvement in the latter case, > > not overall utilization. > > *"Every new cluster requires three new topics, leading to an exponential > > increase in topic creation."* > > You’re absolutely right—the wording here was misleading. The growth is > > linear (three topics per cluster), not exponential. However, it still > > imposes upfront topic creation requirements. > > *"Cross-team dependencies slow down provisioning, delaying deployments."* > > While Kafka Connect can auto-create topics on startup, this is often > > restricted in practice due to: > > 1. ACL rules limiting external topic creation, > > 2. Separate team ownership of topic provisioning, or > > 3. Externally managed Kafka clusters (even if managed by the same team, > > auto-creation is frequently disabled to prevent uncontrolled topic > sprawl). > > Additionally, partition limits on Kafka clusters can constrain the number > > of Connect clusters deployable. > > > > Thanks, > > Pritam > > > > On Tue, Apr 29, 2025 at 2:18 AM Greg Harris <greg.har...@aiven.io.invalid > > > > wrote: > > > > > Hi Pritam, > > > > > > Thanks for the KIP! > > > > > > I'm a little unsure of the motivation here and would appreciate some > more > > > context from your experience. > > > > > > 1. The KIP states "Broker resource utilization is expected to decrease > by > > > approximately 20%, primarily due to reduced partition count and > metadata > > > overhead." Can you share your cluster/connector topology and testing > > method > > > that arrived at this statistic? > > > A Connect cluster's internal topics are amortized among all of the > > > connectors within that cluster, and in a typical deployment, these > > > connectors should be handling at least 10-100x the number of data > > > partitions/bytes as are present in internal topics. So I would not > expect > > > the overhead for internal topics to persistently consume 20% of a Kafka > > > cluster's resources. > > > > > > 2.The KIP states "Every new cluster requires three new topics, leading > to > > > an exponential increase in topic creation." > > > Where are you seeing "exponential" topic creation? It should be linear > in > > > the number of Connect clusters, so I'm wondering if this is an > > unfortunate > > > wording or an exaggeration. > > > > > > 3. The KIP states "Cross-team dependencies slow down provisioning, > > delaying > > > deployments." > > > In my experience provisioning Kafka topics is generally a lightweight > > > operation, and the Connect cluster does it automatically on your behalf > > on > > > first startup. If you are in an environment where additional processes > > are > > > in place that makes topic creation a nuisance, I think that reflects on > > > your environment more than on Connect, and therefore it seems a bit odd > > for > > > Connect to implement a workaround. > > > I am also interested whether other users experience the same nuisance > > > around topic provisioning, and how generally useful this feature is. > > > > > > I think you should also mention some of the technical tradeoffs of this > > > feature, such as: > > > * Read amplification: Connect workers need to consume irrelevant data > > from > > > other clusters and discard it after incurring the costs of transferring > > and > > > deserializing the data, both during startup and ongoing operations. > > > * Security concerns: Sharing credentials for internal topics among > > multiple > > > services violates the Principle of Least Privilege and makes compromise > > of > > > one Connect cluster more impactful. > > > * Correlated failures: The unavailability/corruption of one topic now > > > affects multiple Connect clusters instead of just one > > > * Complications with Exactly Once mode: transactional writes to > internal > > > topics may cause unavailability in other clusters from hanging > > > transactions. > > > > > > Thanks! > > > Greg > > > > > > On Mon, Apr 28, 2025 at 8:01 AM pritam kumar < > kumarpritamm...@gmail.com> > > > wrote: > > > > > > > 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 > > > > > > > > > >