Hi, Xinyu, Thanks for the reply.
JR1. "Under RF=1, we rely on partition movement (reopening the partition via shared storage), which adds a few extra seconds. However, this has a small impact on overall failover latency." Is the added few seconds per partition? What's the latency impact if a broker has thousands of partitions? JR4. "Plugins always make people uneasy, so could we consider a long-term direction of having an abstract Storage Layer with two separate implementations for local disk and cloud storage? " The storage layer is currently designed as a local storage layer below replication. I am not sure if there is an easy way to restructure it to also support cloud storage without replication. The coordinator-based approach may be a cleaner way of supporting cloud storage. Jun On Sun, Nov 9, 2025 at 6:34 PM Xinyu Zhou <[email protected]> wrote: > Hi Jun, > > Thank you so much for your thoughtful feedback—I sincerely apologize for > the delayed response; I somehow missed your email. > > I truly appreciate the insights you’ve shared; they pinpointed several > critical areas that deserve deeper consideration. > > > JR1. One potential downside of using RF=1 is availability. Since there is > > no active standby replica, on a failover, the new leader needs to recover > > the log and rebuild/reload the state before it can serve writes. So, the > > window of unavailability could be large if the producer state is large. > You > > mentioned that your benchmark showed 1-2s leader failover time. What does > > the number look like if there are many clients, each with idempotent > > producer or transactions enabled? This also impacts scalability in the > same > > way. > > Failover typically consists of three phases: Failure Detection, Leader > Switchover, and Client Awareness. Under RF=3, leader election handles the > switchover. Under RF=1, we rely on partition movement (reopening the > partition via shared storage), which adds a few extra seconds. However, > this has a small impact on overall failover latency. > > Anyway, this is indeed a trade-off made by KIP-1183. In a Cloud > environment, compared to an On-premise environment, the probability of > Failure is much lower. This might be a worthwhile trade-off if the benefits > are significant enough. > > And, we haven’t yet encountered large-scale deployments with idempotent > producers enabled—this is something we need to validate. > > > JR2 Another potential downside of RF=1 is that it reduces the opportunity > > for achieving consumer affinity. If a consumer application does > operations > > like joining two topics, having more than 1 read replica enables more > > opportunities for aligning with the consumers. > > Using RF=1 is more about wanting to avoid replicating data at the Kafka > Layer. In fact, object storage makes it easier to scale reads, such as by > adding some read-only replicas. There is no replication semantics; instead, > it leverages the shared properties of object storage to improve fan-out. > > > JR3. Most types of block storage seem to be designed for a single zone > and > > don't provide strong durability and availability. So, it's not clear how > it > > can be used with RF=1. > > Decouple Durability to cloud storage, so Durability is also limited by what > cloud storage provides. Providing Regional durability for block storage is > also an important trend, and among the Top 4 cloud providers, currently > only AWS does not offer it. Additionally, although KIP-1183 does not > currently involve the implementation of the Stream Layer, in AutoMQ's > implementation, Object is still the primary storage (WAL and Data). > However, in some low-latency cases, blocks or files can be used as the > implementation of WAL to provide low latency. > > > JR4. I agree with Satish that it seems there is a lot of work left for > the > > plugin implementer. > > The main idea of KIP-1183 is to first have a relatively abstract Storage > Layer that can support the community in simultaneously iterating on the > classic ISR architecture and the shared storage architecture. This is > because the current storage engine includes a lot of state management done > through local files, which is not very suitable for directly moving to > object storage. > > Plugins always make people uneasy, so could we consider a long-term > direction of having an abstract Storage Layer with two separate > implementations for local disk and cloud storage? This would also avoid the > need to reimplement a large number of Kafka features as in > Coordinator-based solutions. Although it seems to have a relatively high > implementation cost, these changes are mostly confined to the storage > layer, having a smaller impact on the community's iterative work. > > Thank you again for your valuable input, Jun. I really appreciate the depth > of your analysis. > > Wishing you a great day! > > Best regards, > Xinyu > > On Thu, Aug 7, 2025 at 12:53 AM Jun Rao <[email protected]> wrote: > > > Hi, Xinyu, > > > > Thanks for the KIP. A few high level comments. > > > > JR1. One potential downside of using RF=1 is availability. Since there is > > no active standby replica, on a failover, the new leader needs to recover > > the log and rebuild/reload the state before it can serve writes. So, the > > window of unavailability could be large if the producer state is large. > You > > mentioned that your benchmark showed 1-2s leader failover time. What does > > the number look like if there are many clients, each with idempotent > > producer or transactions enabled? This also impacts scalability in the > same > > way. > > > > JR2 Another potential downside of RF=1 is that it reduces the opportunity > > for achieving consumer affinity. If a consumer application does > operations > > like joining two topics, having more than 1 read replica enables more > > opportunities for aligning with the consumers. > > > > JR3. Most types of block storage seem to be designed for a single zone > and > > don't provide strong durability and availability. So, it's not clear how > it > > can be used with RF=1. > > > > JR4. I agree with Satish that it seems there is a lot of work left for > the > > plugin implementer. For example, > > * fencing logic to prevent an old runaway leader from continuing to write > > to the shared storage > > * managing the metadata for shared storage > > * merging smaller objects into bigger ones > > * maintaining a read cache > > This makes it almost impossible for anyone to implement a plugin. > > > > Jun > > > > On Thu, May 15, 2025 at 6:37 PM Xinyu Zhou <[email protected]> wrote: > > > > > Hi Colin, > > > > > > Thank you for taking the time to read this KIP, and no worries, > negative > > > feedback is a catalyst for improvement. > > > > > > Sorry for the inappropriate description in the Motivation section; my > > > background influenced my writing, but I didn’t mean it. I will remove > it. > > > Thanks for the reminder. > > > > > > I completely agree with your point on fragmentation risk. I've seen > many > > > companies maintain their own Kafka fork branches internally, often > > focusing > > > on the storage layer. If the storage layer is more scalable, I think it > > > would help reduce fragmentation. > > > > > > On another note, transitioning Kafka from on-premise to cloud is a > > > long-term process, but we can't ignore cloud needs entirely. Therefore, > > the > > > community may need to support two storage implementations in the > > > foreseeable future, and we should make the storage layer more abstract > to > > > support both. > > > > > > Regarding the relationship between KIP-1183 and 1150, and 1176, as > > > mentioned in the KIP, the architecture of 1150 actually conflicts with > > > Kafka's leader-based architecture. As Jun pointed out, transactions and > > > queues rely on leader-based partitions. How 1150 handles current and > > future > > > features, if they all need to be implemented twice, is a huge burden. > > > > > > For KIP-1176, which I really like, it mainly tries to solve the > > replication > > > traffic cost issue, but doesn't leverage other advantages of shared > > > storage. We can certainly accept KIP-1176, but what's next? We may > still > > > need to discuss how to better support Kafka on cloud storage for > > elasticity > > > and operational advantages. > > > > > > Regarding NFS, yes, Kafka can run on NFS, but it can't utilize NFS's > > shared > > > capabilities. For example, data written by Broker A on NFS can't be > > sensed > > > by Broker B, so even on NFS, reassigning a partition still requires > > > replication. > > > > > > In summary, KIP-1183 aims to discuss how the community views the impact > > of > > > shared storage on the current architecture. Should we embrace it, and > > when? > > > So, I think we should at least reach consensus on these two points: > > > 1. We should consider how to support shared storage, but the community > > > needs to support both local disk and shared storage long-term. > > > 2. Which path should we take? The leaderless architecture of 1150 or > the > > > approach mentioned in 1183. > > > > > > I will update the KIP with our discussion soon. Thanks again for your > > time! > > > > > > Best, > > > Xinyu > > > > > > On Fri, May 16, 2025 at 7:33 AM Colin McCabe <[email protected]> > wrote: > > > > > > > Hi Xinyu Zhou, > > > > > > > > Thanks for the KIP. It's good to see more people contributing to the > > > > community. I think this is your first KIP, so please forgive me for > > > giving > > > > some negative feedback. > > > > > > > > KIPs need to be written in a vendor-neutral manner, for the whole > > > > community. So please do not do things like begin a paragraph with "At > > > > AutoMQ, our goal is..." We really need to focus on the goals of > Apache > > > > Kafka, not the goals of a specific vendor. > > > > > > > > Similarly, it's probably not a good idea to call out all the specific > > > > vendors that have forked Kafka or implemented the Kafka API. We trust > > > that > > > > the work people are contributing to AK is Apache licensed and not > based > > > on > > > > something proprietary, as per our CLA. So we should review the actual > > > > proposed design. > > > > > > > > In the KIP-1150 discussion thread, I called out the pluggable APIs > that > > > > were being proposed as a possible fragmentation risk. I am concerned > > that > > > > the pluggable APIs here could pose an even greater risk. For example, > > if > > > we > > > > end up with a dozen different overlapping AbstractLog > implementations, > > it > > > > will be hard to see that as anything but "disunity." It also means > that > > > it > > > > will be much harder to evolve the core of Kafka. > > > > > > > > After reading this KIP, I'm left confused about what its relationship > > > with > > > > KIP-1150 and KIP-1176 are. The text even states "there are no > rejected > > > > alternatives." But I really disagree with the idea that we can > evaluate > > > > this proposal without understanding its relationship to alternate > > > > proposals. We need to answer the question of why this KIP is > necessary > > if > > > > we have KIP-1150 or KIP-1176. After all, those KIPs come with (small) > > > > pluggable pieces that allow Kafka to hook into multiple blobstores. > > (And > > > > NFS, of course, doesn't need any plugin at all since it exposes a > > > > file-based interface.) So we really need to understand what this KIP > > > brings > > > > to the table. That should go in the "rejected alternatives" section. > > > > > > > > Overall, I would encourage you to propose a concrete design rather > > than a > > > > set of plugin APIs. We cannot really evaluate APIs without > > understanding > > > > the implementation. > > > > > > > > best, > > > > Colin > > > > > > > > > > > > On Tue, May 13, 2025, at 05:21, Xinyu Zhou wrote: > > > > > Dear Kafka Community, > > > > > > > > > > I am proposing a new KIP to introduce a unified shared storage > > > > > solution for Kafka, aiming > > > > > to enhance its scalability and flexibility. This KIP is inspired by > > > > > the ongoing discussions > > > > > around KIP-1150 and KIP-1176, which explore leveraging object > storage > > > > > to achieve cost and > > > > > elasticity benefits. These efforts are commendable, but given the > > > > > widespread adoption of > > > > > Kafka's classic shared-nothing architecture, especially in > on-premise > > > > > environments, we > > > > > need a unified approach that supports a smooth transition from > > > > > shared-nothing to shared > > > > > storage. This KIP proposes refactoring the log layer to support > both > > > > > architectures > > > > > simultaneously, ensuring long-term compatibility and allowing Kafka > > to > > > > > fully leverage > > > > > shared storage services like S3, HDFS, and NFS. > > > > > > > > > > The core of this proposal includes introducing abstract log and log > > > > > segment classes and a > > > > > new 'Stream' API to bridge the gap between shared storage services > > and > > > > > Kafka's storage > > > > > layer. This unified solution will enable Kafka to evolve while > > > > > maintaining backward > > > > > compatibility, supporting both on-premise and cloud deployments. I > > > > > believe this approach > > > > > is crucial for Kafka's continued success and look forward to your > > > > > thoughts and feedback. > > > > > > > > > > > > > > > Link to the KIP for more details: > > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1183%3A+Unified+Shared+Storage > > > > > > > > > > Best regards, > > > > > > > > > > Xinyu > > > > > > > > > >
