Hi Luke,

Thank you for your review and insightful questions.

1. Regarding the future plan for this KIP, once we establish an abstract
log layer, we might implement a default version on object storage for
Kafka. For other types of shared storage, we'll decide based on community
needs.

2. In my view, I recommend having a Stream layer in Kafka. We need an
abstraction between Log and shared storage mediums to truly achieve the
Unified concept mentioned in the KIP title. However, if the community
prefers a narrower scope, making it optional is also acceptable.

3. For 3.a, 3.b, and 3.d, more details are needed, and I require additional
time to enhance these sections. Regarding 3.c, in a future Kafka cluster
with a shared-storage-only setup, brokers won't need local data disks as
all topics will be stored in a shared log. But only topics with consensus
algorithms (like ISR and Raft) can store metadata, so storing metadata in
an internal topic isn't possible in a shared-storage architecture. Do you
have any concerns about storing the metadata in KRaft?

Regards,
Xinyu

On Wed, May 14, 2025 at 3:22 PM Luke Chen <show...@gmail.com> wrote:

> Hi Xinyu,
>
> Thanks for the proposal!
> I agree abstracting the log and logSegment layers can allow users to
> implement any kinds of storage mediums.
>
> Some high-level comments:
> 1. I expect to see the future plan of this KIP, but can't find it.
> What do you expect to achieve the "shared storage" goal after the Unified
> Shared Storage KIP?
> Maybe "Replication protocol API change" for phase 2? Maybe "Useful tools to
> interact with shared storage" for next phase? I don't know, but just want
> to know what is the full design in your thought?
>
> 2. Why is the "Stream layer" optional? Currently, in each LogSegment, it
> contains FileRecords, indexes, producer snapshots,... etc implementation in
> the file-based storage. After this KIP, there is no implementation for
> SharedLogSegment, right? If we don't have Stream Layer, how do users rely
> on the sharedLog abstraction to develop for their own storage?
>
> 3. Following (2), because we abstract the log layer now, we need all the
> detailed information to instruct users to implement what they desire.
> Suppose that is the Stream Layer, I expect there will be more detailed
> explanation about it, like:
> (a) What are the methods in the interface doing? When will they be invoked?
> Why ...
> (b) When and how will the interface interact with shareLogSegment? Like
> when will the meta stream come into play?
> (c) What is the motivation about storing the metadata in KRaft, instead of
> another internal topic or creating another interface or something else?
> (d) The detail of messages, i.e. GET_KVS, PUT_KVS, ... etc.
>
>
> Thank you.
> Luke
>
> On Tue, May 13, 2025 at 8:22 PM Xinyu Zhou <yu...@apache.org> 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