Luke, Thanks for your comments, see my answers below inline. On Thursday, May 1, 2025 at 03:20:54 AM PDT, Luke Chen <show...@gmail.com> wrote: Hi Henry,
This is a very interesting proposal! I love the idea to minimize the code change to be able to quickly get delivered. Thanks for proposing this! Some questions: 1. In this KIP, we add one more tier of storage. That is: local disk -> fast object store -> slow object store. Why can't we allow users to replace the local disk with the fast object store directly? Any consideration on this? If we don't have the local disk, the follower fetch will be much simplified without downloading from the fast object store, is my understanding correct? HC> The fast object storage is not as fast as local disk, the data latency on fast object storage is going to be in 10ms for big data packets and the local disk append is fast since we only need to append the records into the page cache of the local file (the flush from page cache to disk is done asynchronously without affecting the main request/reply cycle between producer and leader broker). This is actually the major difference between this KIP and KIP-1150, although KIP-1150 can completely removing the local disk but they are going to have a long latency (their main use cases is for customer can tolerate 200ms latency) and they need to start build their own memory management and caching strategy since they are not using page cache anymore. Our KIP has no latency change (comparing the current Kafka status) on acks=1 path which I believe is still the operating mode for many company's logging pipelines. 2. Will the WALmetadata be deleted after the data in fast object storage is deleted? I'm a little worried about the metadata size in the WALmetadata. I guess the __remote_log_metadata topic is stored in local disk only, right? HC> Currently we are reusing the classes and constructs from KIP-405, e.g. the __remote_log_metadata topic and ConsumerManager and ProducerManager. As you pointed out the size of segments from active log segments is going to be big, our vision is to create a separate metadata topic for active log segments then we can have a shorter retention setting for this topic to remove the segment metadata faster, but we would need to refactor code in ConsumerManager and ProducerManager to work with 2nd metadata topic. 3. In this KIP, we assume the fast object store is different from the slow object store. Is it possible we allow users to use the same one? Let's say, we set both fast/slow object store = S3 (some use cases doesn't care about too much on the latency), if we offload the active log segment onto fast object store (S3), can we not offload the segment to slow object store again after the log segment is rolled? I'm thinking if it's possible we learn(borrow) some ideas from KIP-1150? This way, we can achieve the similar goal since we accumulate (combine) data in multiple partitions and upload to S3 to save the cost. HC> Of course people can choose just to use S3 for both fast and slow object storage. They can have the same class implementing both RemoteStorageManager and RemoteWalStorageManager, we proposed RemoteWalStorageManager as a separate interface to give people different implementation choices. I think KIP-1176 (this one) and KIP-1150 can combine some ideas or implementations. We mainly focus on cutting AZ transfer cost while maintaining the same performance characteristics (such as latency) and doing a smaller evolution of the current Kafka code base. KIP-1150 is a much ambitious effort with a complete revamp of Kafka storage and memory management system. Thank you. Luke On Thu, May 1, 2025 at 1:45 PM Henry Haiying Cai <haiying_...@yahoo.com.invalid> wrote: > Link to the KIP: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-1176%3A+Tiered+Storage+for+Active+Log+Segment > Motivation > In KIP-405, the community has proposed and implemented the tiered storage > for old Kafka log segment files, when the log segments is older than > local.retention.ms, it becomes eligible to be uploaded to cloud's object > storage and removed from the local storage thus reducing local storage > cost. KIP-405 only uploads older log segments but not the most recent > active log segments (write-ahead logs). Thus in a typical 3-way replicated > Kafka cluster, the 2 follower brokers would still need to replicate the > active log segments from the leader broker. It is common practice to set up > the 3 brokers in three different AZs to improve the high availability of > the cluster. This would cause the replications between leader/follower > brokers to be across AZs which is a significant cost (various studies show > the across AZ transfer cost typically comprises 50%-60% of the total > cluster cost). Since all the active log segments are physically present on > three Kafka Brokers, they still comprise significant resource usage on the > brokers. The state of the broker is still quite big during node > replacement, leading to longer node replacement time. KIP-1150 recently > proposes diskless Kafka topic, but leads to increased latency and a > significant redesign. In comparison, this proposed KIP maintains identical > performance for acks=1 producer path, minimizes design changes to Kafka, > and still slashes cost by an estimated 43%. >