Hi Satish, Thanks for your incisive comments!
For SD-1: I am now confident that we have a better chance of reaching a consensus on the abstract layer. I will complete this part and address all the questions from you and Luke. Once finished, I will ping you again. For SD-2: In this KIP, a partition consists of several streams, but most of the partition's metadata (including producer snapshots) is stored in a special stream called MetaStream. Therefore, we only need to store the relationship between a partition and its MetaStream. For each partition, we need to store approximately 28 bytes: 16 for the topic ID, 4 for the partition ID, and 8 for the stream ID. With 100,000 partitions, we require less than 3 MiB of metadata space. For SD-3: Yes, RF should always be set to 1 for shared storage. Since we have decoupled durability to the storage service, data replication is no longer necessary. Additionally, we need to ensure that each write to shared storage is written through, meaning we should provide synchronous write semantics. For now, there is no need to alter other replication semantics. In our benchmark for SD-4, using S3 as shared storage allows us to change the leader of a specific partition in just 1-2 seconds. We simply close the partition on node-a and open it on node-b, recovering the partition metadata from S3 with a single S3 get API call to load the metadata from MetaStream. You pointed out a gap in the KIP; we do need to refine the partition reassignment process to accommodate shared storage, but it's not complex. For SD-5: Yes, you're right. Even with everything mentioned in the KIP, implementing `Stream` on a specific shared storage service is still challenging. That's why we want to move Kafka to a shared storage architecture step by step. Maybe, we can also provide default batch, cache, and read-ahead mechanisms to reduce the complexity of stream implementation. Thanks again for your time. Regards, Xinyu On Wed, May 14, 2025 at 7:11 PM Satish Duggana <satish.dugg...@gmail.com> wrote: > Thanks Xinyu for accepting our request to initiate a KIP on AutoMq's > proposal. > > SD-1: It is a good start towards building log storage layer > abstractions, including LogSegment and Log, for both local and shared > storage. However, more clarity is needed on how these abstractions are > defined. The classes currently encapsulate a significant amount of > functionality. It's also important to identify the necessary APIs, > explain their lifecycle, and detail how they integrate with the > internal storage module and other dependencies. > > SD-2: How much metadata overhead will be published to KRaft? > > SD-3: How would it affect replication protocol? It seems RF becomes 1 > as no data needs to be replicated but are there any other replication > semantic changes required? > > SD-4: How long does it take another node to become a loader for a > partition as the metadata may need to be loaded on the new node? > > SD-5: In this approach, much of the core functionality resides within > the Stream implementation. It might be worth considering a design > where the building blocks handle most of the complexity, allowing the > plugin to remain as lightweight as possible. > > Thanks, > Satish. > > On Tue, 13 May 2025 at 17:52, 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 >