Hi Greg, Thanks for this KIP! It is obviously very ambitious, but it's great to have a conversation about it.
I'll start with some general points: Do you have a plan in mind for how to proceed with elaborating this KIP? While I like how you're involving the community in elaborating the KIP, I think there is a danger, which is more likely with this inclusive approach, in trying to attempt too much at once. In my opinion someone needs to take the difficult decisions necessary to limit the initial scope (and, just as importantly, communicate that clearly) in order to maximise the chances of actually getting something accepted and implemented. Can we assume that you're that person? Defining the what and how of the metadata replication, and the log replication seem to me to be the core of what you're trying to achieve here. We should make anything that is not crucial to that (i.e. NAT punching) a non-goal of this KIP. Future KIPs can easily add those features. I also had a few specific points: Motivation M1. I don't find the "logical replication" vs "physical replication" particularly helpful. I think one key property is "offset preserving", which is also self-explanatory. Slightly more generally, we could define the concept of "consumer transparency", i.e. a consumer could reconnect to either cluster and observe the same sequences of records (same order, same offsets, and same transaction visibility). Consumer transparency requires synchronous replication, but offset preserving does not. M2. In the motivation you mention that MM offers asynchronous replication, but the Goals subsection doesn't mention support for synchronous replication. We should be clear which (or both) we're aiming for. M3. A Non-Goals section would be useful, especially for a KIP that's large and ambitious like this one. M4. It might also be worth having a list of Assumptions. Here we could list all the things we want to assume in order to make the initial KIP feasible. M5. For example we should be explicit about whether or not it is assumed that the same people are operating (and thus have visibility into) both clusters. M6. One thing worth calling out is whether the clusters themselves are in a leader/follower relationship (e.g. the DR scenario), or whether this is a topic-level concern. I guess it's topic level from the topic and consumer group regexes. But this has consequences we should explore. For example what if a transaction includes records in topics X and Y, where X is replicated but Y is not? M7. I think you should be clear about whether this leader/follower relationship can be reversed, and in what circumstances. In the user interface section you talk about "disconnected", but not this kind of fail-back. User interface U1. "Links can be temporarily or permanently disconnected." Are you describing a fact about the network between the two clusters, or is this disconnection something actively managed by the system, or by the operator? Data semantics D1. The KIP says "both cross-cluster topics and intra-cluster replicas: Have the same configuration as their source" but you also say "cross-cluster replicas: Have a separate topic-id", this seems like a contradiction, on the face of it. It seems like there's a whole host of devils in the detail behind this. It implies replication of (some of) the __cluster_metadata, I think, but not all (since you say ACLs are not replicated). If that's right, then what does it imply about referential integrity between metadata records? i.e. what if metadata record A (which is replicated) references record B (which is not)? Even if this is not possible by design initially, how does it constrain the future evolution of metadata record schemas? Is any such metadata replication going to be transaction preserving? If the topic ids can differ then what is responsible for the mapping and rewriting of metadata records which include topic ids? D2. "The network path between Kafka clusters is assumed to be less reliable than the intra-cluster network," we should be explicit about whether or not we're assuming similar network latencies and bandwidth for the inter-cluster network links as for the in-cluster ones. D3 "Are not eligible for fetch-from-follower on the source cluster" the reason for this isn't immediately apparent to me. Thanks again, Tom On Tue, 3 Oct 2023 at 09:37, Greg Harris <greg.har...@aiven.io.invalid> wrote: > Hi all, > > I've opened an extremely early draft of the Cross-Cluster Replication > feature, and I would like to invite any and all co-authors to expand > on it. Find the page here: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-986%3A+Cross-Cluster+Replication > > This is not strictly an invitation to "review" the KIP, as the > document has much less detail than other KIPs of similar complexity. > But if you are knowledgeable in this area, some early sanity checks > would be greatly appreciated. > > I've included a "shopping list" of properties that appear to me to be > desirable, but I don't have an implementation in mind that meets these > requirements. If you have additional requirements, an alternative UX > in mind, or wish to propose some implementation details, please edit > the KIP with your contributions. > > Thanks everyone! > > Greg Harris > Aiven, Inc > >