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 <cmcc...@apache.org> 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 >