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