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

Reply via email to