Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2021-01-09 Thread Satish Duggana
Hi Jun, Thanks for your comments. Please find the inline replies below. 6022. For packages used for server plugins, the convention is to use org.apache.kafka.server. See java-based Authorizer as an example. Sure, ‘org.apache.kafka.common.log.remote.storage’ renamed to

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2021-01-09 Thread Satish Duggana
Hi Kowshik, Thanks for your comments. Please find the inline replies below. 9002. Under the "Upgrade" section, the configuration mentioned is 'remote.log.storage.system.enable'. However, under "Public Interfaces" section the corresponding configuration is 'remote.storage.system.enable'. Could we

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-16 Thread Jun Rao
Hi, Satish, Thanks for the reply. A few more followup comments. 6022. For packages used for server plugins, the convention is to use org.apache.kafka.server. See java-based Authorizer as an example. 9100. Do we need DeletePartitionStateRecord in flat_file_format? The flat file captures the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-15 Thread Kowshik Prakasam
Hi Satish, Thanks for the updates! A few more comments below. 9001. Under the "Upgrade" section, there is a line mentioning: "Upgrade the existing Kafka cluster to 2.7 version and allow this to run for the log retention of user topics that you want to enable tiered storage. This will allow all

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-14 Thread Satish Duggana
Hi Jun, Thanks for your comments. Please go through the inline replies. 5102.2: It seems that both positions can just be int. Another option is to have two methods. Would it be clearer? InputStream fetchLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata, int startPosition)

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-07 Thread Jun Rao
Hi, Satish, Thanks for the reply. A few more comments below. 5102.2: It seems that both positions can just be int. Another option is to have two methods. Would it be clearer? InputStream fetchLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-07 Thread Satish Duggana
Hi Jun, Thanks for your comments. Please find the inline replies below. >605.2 It's rare for the follower to need the remote data. So, the current approach is fine too. Could you document the process of rebuilding the producer state since we can't simply trim the producerState to an offset in the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-02 Thread Jun Rao
Hi, Satish, Thanks for the updated KIP. A few more comments below. 605.2 It's rare for the follower to need the remote data. So, the current approach is fine too. Could you document the process of rebuilding the producer state since we can't simply trim the producerState to an offset in the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-12-01 Thread Satish Duggana
Hi, We updated the KIP with the points mentioned in the earlier mail except for KIP-516 related changes. You can go through them and let us know if you have any comments. We will update the KIP with the remaining todo items and KIP-516 related changes by end of this week(5th Dec). Thanks, Satish.

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-20 Thread Kowshik Prakasam
Hi Harsha/Satish, Hope you are doing well. Would you be able to please update the meeting notes section for the most recent 2 meetings (from 10/13 and 11/10)? It will be useful to share the context with the community.

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-10 Thread Kowshik Prakasam
Hi Harsha, The goal we discussed is to aim for preview in AK 3.0. In order to get us there, it will be useful to think about the order in which the code changes will be implemented, reviewed and merged. Since you are driving the development, do you want to layout the order of things? For example,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-10 Thread Harsha Chintalapani
Thanks Kowshik for the link. Seems reasonable, as we discussed on the call, code and completion of this KIP will be taken up by us. Regarding Milestone 2, what you think it needs to be clarified there? I believe what we are promising in the KIP along with unit tests, systems tests will be

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-10 Thread Kowshik Prakasam
Hi Harsha/Satish, Thanks for the discussion today. Here is a link to the KIP-405 development milestones google doc we discussed in the meeting today: https://docs.google.com/document/d/1B5_jaZvWWb2DUpgbgImq0k_IPZ4DWrR8Ru7YpuJrXdc/edit . I have shared it with you. Please have a look and share your

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-10 Thread Satish Duggana
Hi Jun, Thanks for your comments. Please find the inline replies below. 605.2 "Build the local leader epoch cache by cutting the leader epoch sequence received from remote storage to [LSO, ELO]." I mentioned an issue earlier. Suppose the leader's local start offset is 100. The follower finds a

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-06 Thread Kowshik Prakasam
Hi Satish, Thanks for your response. 5015. That makes sense, good point. 5019 and 5020. My 2 cents is that while you are implementing it, it will be useful to update the KIP with details about the RocksDB-based design that you envision. This will facilitate the discussions. Cheers, Kowshik

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-06 Thread Satish Duggana
Hi Kowshik, Thanks for your comments. 5012. In the RemoteStorageManager interface, there is an API defined for each file type. For example, fetchOffsetIndex, fetchTimestampIndex etc. To avoid the duplication, I'd suggest we can instead have a FileType enum and a common get API based on the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-11-04 Thread Jun Rao
Hi, Satish, Thanks for the updated KIP. A few more comments below. 605.2 "Build the local leader epoch cache by cutting the leader epoch sequence received from remote storage to [LSO, ELO]." I mentioned an issue earlier. Suppose the leader's local start offset is 100. The follower finds a remote

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-10-27 Thread Kowshik Prakasam
Hi Satish, Thanks for the updates to the KIP. Here are my first batch of comments/suggestions on the latest version of the KIP. 5012. In the RemoteStorageManager interface, there is an API defined for each file type. For example, fetchOffsetIndex, fetchTimestampIndex etc. To avoid the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-10-26 Thread Satish Duggana
Hi, KIP is updated with 1) topic deletion lifecycle and its related items 2) Protocol changes(mainly related to ListOffsets) and other minor changes. Please go through them and let us know your comments. Thanks, Satish. On Mon, Sep 28, 2020 at 9:10 PM Satish Duggana wrote: > > Hi Dhruvil, >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-28 Thread Satish Duggana
Hi Dhruvil, Thanks for looking into the KIP and sending your comments. Sorry for the late reply, missed it in the mail thread. 1. Could you describe how retention would work with this KIP and which threads are responsible for driving this work? I believe there are 3 kinds of retention processes

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-16 Thread Dhruvil Shah
Hi Satish, Harsha, Thanks for the KIP. Few questions below: 1. Could you describe how retention would work with this KIP and which threads are responsible for driving this work? I believe there are 3 kinds of retention processes we are looking at: (a) Regular retention for data in tiered

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-07 Thread Harsha Ch
Hi All, We are all working through the last meeting feedback. I'll cancel the tomorrow 's meeting and we can meanwhile continue our discussion in mailing list. We can start the regular meeting from next week onwards. Thanks, Harsha On Fri, Sep 04, 2020 at 8:41 AM, Satish Duggana <

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-09-04 Thread Satish Duggana
Hi Jun, Thanks for your thorough review and comments. Please find the inline replies below. 600. The topic deletion logic needs more details. 600.1 The KIP mentions "The controller considers the topic partition is deleted only when it determines that there are no log segments for that topic

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-31 Thread Jun Rao
Hi, Satish, Thanks for the updated KIP. Made another pass. A few more comments below. 600. The topic deletion logic needs more details. 600.1 The KIP mentions "The controller considers the topic partition is deleted only when it determines that there are no log segments for that topic partition

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-31 Thread Satish Duggana
KIP is updated with - Remote log segment metadata topic message format/schema. - Added remote log segment metadata state transitions and explained how the deletion of segments is handled, including the case of partition deletions. - Added a few more limitations in the "Non goals" section. Thanks,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-26 Thread Harsha Ch
Updated the KIP with Meeting Notes section https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes On Tue, Aug 25, 2020 at 1:03 PM Jun Rao wrote: > Hi, Harsha, > > Thanks for the summary. Could you add the summary and the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-25 Thread Jun Rao
Hi, Harsha, Thanks for the summary. Could you add the summary and the recording link to the last section of https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals ? Jun On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani wrote: > Thanks everyone for attending the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-25 Thread Harsha Chintalapani
Thanks everyone for attending the meeting today. Here is the recording https://drive.google.com/file/d/14PRM7U0OopOOrJR197VlqvRX5SXNtmKj/view?usp=sharing Notes: 1. KIP is updated with follower fetch protocol and ready to reviewed 2. Satish to capture schema of internal metadata topic in

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-24 Thread Ying Zheng
We did some basic feature tests at Uber. The test cases and results are shared in this google doc: https://docs.google.com/spreadsheets/d/1XhNJqjzwXvMCcAOhEH0sSXU6RTvyoSf93DHF-YMfGLk/edit?usp=sharing The performance test results were already shared in the KIP last month. On Mon, Aug 24, 2020 at

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-24 Thread Harsha Ch
"Understand commitments towards driving design & implementation of the KIP further and how it aligns with participant interests in contributing to the efforts (ex: in the context of Uber’s Q3/Q4 roadmap)." What is that about? On Mon, Aug 24, 2020 at 11:05 AM Kowshik Prakasam wrote: > Hi Harsha,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-24 Thread Kowshik Prakasam
Hi Harsha, The following google doc contains a proposal for temporary agenda for the KIP-405 sync meeting tomorrow: https://docs.google.com/document/d/1pqo8X5LU8TpwfC_iqSuVPezhfCfhGkbGN2TqiPA3LBU/edit . Please could you add it to the Google

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-20 Thread Alexandre Dupriez
Hi Jun, Many thanks for your initiative. If you like, I am happy to attend at the time you suggested. Many thanks, Alexandre Le mer. 19 août 2020 à 22:00, Harsha Ch a écrit : > Hi Jun, > Thanks. This will help a lot. Tuesday will work for us. > -Harsha > > > On Wed, Aug 19, 2020 at

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-19 Thread Harsha Ch
Hi Jun, Thanks. This will help a lot. Tuesday will work for us. -Harsha On Wed, Aug 19, 2020 at 1:24 PM Jun Rao wrote: > Hi, Satish, Ying, Harsha, > > Do you think it would be useful to have a regular virtual meeting to > discuss this KIP? The goal of the meeting will be sharing >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-19 Thread Jun Rao
Hi, Satish, Ying, Harsha, Do you think it would be useful to have a regular virtual meeting to discuss this KIP? The goal of the meeting will be sharing design/development progress and discussing any open issues to accelerate this KIP. If so, will every Tuesday (from next week) 9am-10am PT work

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-18 Thread Satish Duggana
Hi Kowshik, Thanks for looking into the KIP and sending your comments. 5001. Under the section "Follower fetch protocol in detail", the next-local-offset is the offset upto which the segments are copied to remote storage. Instead, would last-tiered-offset be a better name than

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-13 Thread Kowshik Prakasam
Hi Harsha/Satish, Thanks for the great KIP. Below are the first set of questions/suggestions I had after making a pass on the KIP. 5001. Under the section "Follower fetch protocol in detail", the next-local-offset is the offset upto which the segments are copied to remote storage. Instead, would

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-08-06 Thread Satish Duggana
Hi Jun, Thanks for your comments. > At the high level, that approach sounds reasonable to me. It would be useful to document how RLMM handles overlapping archived offset ranges and how those overlapping segments are deleted through retention. Sure, we will document that in the KIP. >How is the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-31 Thread Jun Rao
Hi, Ying, Thanks for the reply. 1001. Using the new leader as the source of truth may be fine too. What's not clear to me is when a follower takes over as the new leader, from which offset does it start archiving to the block storage. I assume that the new leader starts from the latest archived

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Ying Zheng
1001. We did consider this approach. The concerns are 1) This makes unclean-leader-election rely on remote storage. In case the remote storage is unavailable, Kafka will not be able to finish the unclean-leader-election. 2) Since the user set local retention time (or local retention bytes), I

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Jun Rao
Hi, Satish, Thanks for the reply. 1001. In your example, I was thinking that you could just download the latest leader epoch from the object store. After that you know the leader should end with offset 1100. The leader will delete all its local data before offset 1000 and start accepting new

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-28 Thread Satish Duggana
HI Jun, Thanks for your comments. We put our inline replies below. 1001. I was thinking that you could just use the tiered metadata to do the reconciliation. The tiered metadata contains offset ranges and epoch history. Those should be enough for reconciliation purposes. If we use remote storage

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-23 Thread Jun Rao
Hi, Satish, Thanks for the reply. A few quick comments below. 1001. I was thinking that you could just use the tiered metadata to do the reconciliation. The tiered metadata contains offset ranges and epoch history. Those should be enough for reconciliation purposes. 1003.3 Having just a

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-23 Thread Satish Duggana
Hi Jun, Thank you for the comments! Ying, Harsha and I discussed and put our comments below. 1001. The KIP described a few scenarios of unclean leader elections. This is very useful, but I am wondering if this is the best approach. My understanding of the proposed approach is to allow the new

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-20 Thread Jun Rao
Hi, Satish, Ying, Harsha, Thanks for the updated KIP. A few more comments below. 1000. Regarding Colin's question on querying the metadata directly in the remote block store. One issue is that not all block stores offer the needed api to query the metadata. For example, S3 only offers an api to

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-15 Thread Harsha Ch
Hi Colin, Thats not what we said in the previous email. RLMM is pluggable storage and by running numbers even 1PB data you do not need more than 10GB local storage. If in future this becomes a blocker for any users we can revisit but this does not warrant another implementation at

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-15 Thread Colin McCabe
Hi Ying, Thanks for the response. It sounds like you agree that storing the metadata in the remote storage would be a better design overall. Given that that's true, is there any reason to include the worse implementation based on RocksDB? Choosing a long-term metadata store is not something

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-14 Thread Ying Zheng
Hi Jun, Hi Colin, Satish and I are still discussing some details about how to handle transactions / producer ids. Satish is going to make some minor changes to RLMM API and other parts. Other than that, we have finished updating the KIP I agree with Colin that the current design of using rocksDB

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-12 Thread Satish Duggana
We already mentioned in the KIP that RemoteLogMetadataMamnager is pluggable. Users have an option to plugin their own implementation of RLMM instead of using the default implementaion(which is on topic storage), which can be based on their remote storage environments like AWS, GCP, Azure etc. On

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-12 Thread Satish Duggana
Hi Colin, Thanks for looking into the KIP. I guess you are talking about the option mentioned for a cache used in the default implementation of RemoteLogMetadataManager. RocksDB is put as an option of cache for the initial version of the default implementation because it is easy and works with

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-11 Thread ignacio gioya
Hi, i wan to unsubscribe from this list. Can i do it? Please :) Thank u! Regards!! El sáb., 11 jul. 2020 a las 22:06, Adam Bellemare () escribió: > My 2 cents - > > I agree with Colin. I think that it's important that the metadata not grow > unbounded without being delegated to external

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-11 Thread Adam Bellemare
My 2 cents - I agree with Colin. I think that it's important that the metadata not grow unbounded without being delegated to external storage. Indefinite long-term storage of entity data in Kafka can result in extremely large datasets where the vast majority of data is stored in the external

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-10 Thread Colin McCabe
Hi all, Thanks for the KIP. I took a look and one thing that stood out to me is that the more metadata we have, the more storage we will need on local disk for the rocksDB database. This seems like it contradicts some of the goals of the project. Ideally the space we need on local disk

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-09 Thread Harsha Chintalapani
Hi Jun, Thanks for the replies and feedback on design and giving input. We are coming close to finish the implementation. We also did several perf tests as well at our peak production loads and with tiered storage we didn't see any degradation on write throughputs and latencies. Ying

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-07 Thread Jun Rao
Hi, Ying, Thanks for the update. It's good to see the progress on this. Please let us know when you are done updating the KIP wiki. Jun On Tue, Jul 7, 2020 at 10:13 AM Ying Zheng wrote: > Hi Jun, > > Satish and I have added more design details in the KIP, including how to > keep consistency

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-07 Thread Ying Zheng
Hi Jun, Satish and I have added more design details in the KIP, including how to keep consistency between replicas (especially when there is leadership changes / log truncations) and new metrics. We also made some other minor changes in the doc. We will finish the KIP changes in the next couple

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-07-07 Thread Jun Rao
Hi, Satish, Harsha, Any new updates on the KIP? This feature is one of the most important and most requested features in Apache Kafka right now. It would be helpful if we can make sustained progress on this. Could you share how far along is the design/implementation right now? Is there anything

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-06-09 Thread Satish Duggana
We did not want to add many implementation details in the KIP. But we decided to add them in the KIP as appendix or sub-sections(including follower fetch protocol) to describe the flow with the main cases. That will answer most of the queries. I will update on this mail thread when the respective

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-06-06 Thread Alexandre Dupriez
Hi Satish, A couple of questions specific to the section "Follower Requests/Replication", pages 16:17 in the design document [1]. 900. It is mentioned that followers fetch auxiliary states from the remote storage. 900.a Does the consistency model of the external storage impacts reads of leader

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-06-04 Thread Satish Duggana
Hi Jun, Please let us know if you have any comments on "transactional support" and "follower requests/replication" mentioned in the wiki. Thanks, Satish. On Tue, Jun 2, 2020 at 9:25 PM Satish Duggana wrote: > > Thanks Jun for your comments. > > >100. It would be useful to provide more details

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-06-02 Thread Satish Duggana
Thanks Jun for your comments. >100. It would be useful to provide more details on how those apis are used. >Otherwise, it's kind of hard to really assess whether the new apis are >sufficient/redundant. A few examples below. We will update the wiki and let you know. >100.1 deleteRecords seems

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-28 Thread Jun Rao
Hi, Satish, Made another pass on the wiki. A few more comments below. 100. It would be useful to provide more details on how those apis are used. Otherwise, it's kind of hard to really assess whether the new apis are sufficient/redundant. A few examples below. 100.1 deleteRecords seems to only

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-28 Thread Satish Duggana
Hi Jun, Gentle reminder. Please go through the updated wiki and let us know your comments. Thanks, Satish. On Tue, May 19, 2020 at 3:50 PM Satish Duggana wrote: > Hi Jun, > Please go through the wiki which has the latest updates. Google doc is > updated frequently to be in sync with wiki. > >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-19 Thread Satish Duggana
Hi Jun, Please go through the wiki which has the latest updates. Google doc is updated frequently to be in sync with wiki. Thanks, Satish. On Tue, May 19, 2020 at 12:30 AM Jun Rao wrote: > Hi, Satish, > > Thanks for the update. Just to clarify. Which doc has the latest updates, > the wiki or

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-18 Thread Jun Rao
Hi, Satish, Thanks for the update. Just to clarify. Which doc has the latest updates, the wiki or the google doc? Jun On Thu, May 14, 2020 at 10:38 AM Satish Duggana wrote: > Hi Jun, > Thanks for your comments. We updated the KIP with more details. > > >100. For each of the operations

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-17 Thread Alexandre Dupriez
Hi Satish, Thank you for your updates. I have some questions around potential use cases when unclean leader election is enabled. It is possible that a range of offsets of a segment which is already offloaded to a tier storage is included in the range of offsets to be truncated. A follower,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-05-14 Thread Satish Duggana
Hi Jun, Thanks for your comments. We updated the KIP with more details. >100. For each of the operations related to tiering, it would be useful to provide a description on how it works with the new API. These include things like consumer fetch, replica fetch, offsetForTimestamp, retention

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-02-26 Thread Harsha Chintalapani
On Tue, Feb 25, 2020 at 12:46 PM, Jun Rao wrote: > Hi, Satish, > > Thanks for the updated doc. The new API seems to be an improvement > overall. A few more comments below. > > 100. For each of the operations related to tiering, it would be useful to > provide a description on how it works with

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-02-25 Thread Jun Rao
Hi, Satish, Thanks for the updated doc. The new API seems to be an improvement overall. A few more comments below. 100. For each of the operations related to tiering, it would be useful to provide a description on how it works with the new API. These include things like consumer fetch, replica

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-02-24 Thread Satish Duggana
Hi Jun, Please look at the earlier reply and let us know your comments. Thanks, Satish. On Wed, Feb 12, 2020 at 4:06 PM Satish Duggana wrote: > > Hi Jun, > Thanks for your comments on the separation of remote log metadata > storage and remote log storage. > We had a few discussions since early

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2020-02-12 Thread Satish Duggana
Hi Jun, Thanks for your comments on the separation of remote log metadata storage and remote log storage. We had a few discussions since early Jan on how to support eventually consistent stores like S3 by uncoupling remote log segment metadata and remote log storage. It is written with details in

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-27 Thread Ivan Yurchenko
Hi all, Jun: > (a) Cost: S3 list object requests cost $0.005 per 1000 requests. If you > have 100,000 partitions and want to pull the metadata for each partition at > the rate of 1/sec. It can cost $0.5/sec, which is roughly $40K per day. I want to note here, that no reasonably durable storage

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-19 Thread Alexandre Dupriez
Hi Jun, Thank you for the feedback. I am trying to understand how a push-based approach would work. In order for the metadata to be propagated (under the assumption you stated), would you plan to add a new API in Kafka to allow the metadata store to send them directly to the brokers? Thanks,

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-18 Thread Jun Rao
Hi, Satish and Ivan, There are different ways for doing push based metadata propagation. Some object stores may support that already. For example, S3 supports events notification ( https://docs.aws.amazon.com/AmazonS3/latest/dev/NotificationHowTo.html). Otherwise one could use a separate metadata

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-18 Thread Alexandre Dupriez
Hi all, A.1 As commented on PR 7561, S3 consistency model [1][2] implies RSM cannot relies solely on S3 APIs to guarantee the expected strong consistency. The proposed implementation [3] would need to be updated to take this into account. Let’s talk more about this. A.2 Contract for the RSM API

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-05 Thread Satish Duggana
Hi Jun, Thanks for your reply. Currently, `listRemoteSegments` is called at the configured interval(not every second, defaults to 30secs). Storing remote log metadata in a strongly consistent store for S3 RSM is raised in PR-comment[1]. RLM invokes RSM at regular intervals and RSM can give remote

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-05 Thread Ivan Yurchenko
Hi Jun and others. Jun, All these are really valid concerns. Probably we should think about backing implementations like S3 with a metadata storage whose consistency model and pricing is better that pure S3, maybe even a Kafka topic (I guess this might be something you refer to as push-based

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-04 Thread Jun Rao
Hi, Harsha, Thanks for the reply. 40/41. I am curious which block storages you have tested. S3 seems to be one of the popular block stores. The concerns that I have with pull based approach are the following. (a) Cost: S3 list object requests cost $0.005 per 1000 requests. If you have 100,000

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-12-03 Thread Harsha Chintalapani
Hi Jun, Thanks for the reply. On Tue, Nov 26, 2019 at 3:46 PM, Jun Rao wrote: > Hi, Satish and Ying, > > Thanks for the reply. > > 40/41. There are two different ways that we can approach this. One is what > you said. We can have an opinionated way of storing and populating the tier >

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-26 Thread Jun Rao
Hi, Satish and Ying, Thanks for the reply. 40/41. There are two different ways that we can approach this. One is what you said. We can have an opinionated way of storing and populating the tier metadata that we think is good enough for everyone. I am not sure if this is the case based on what's

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-19 Thread Ying Zheng
Please ignore my previous email I didn't know Apache requires all the discussions to be "open" On Tue, Nov 19, 2019, 5:40 PM Ying Zheng wrote: > Hi Jun, > > Thank you very much for your feedback! > > Can we schedule a meeting in your Palo Alto office in December? I think a > face to face

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-19 Thread Ying Zheng
Hi Jun, Thank you very much for your feedback! Can we schedule a meeting in your Palo Alto office in December? I think a face to face discussion is much more efficient than emails. Both Harsha and I can visit you. Satish may be able to join us remotely. On Fri, Nov 15, 2019 at 11:04 AM Jun Rao

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-19 Thread Ying Zheng
On Fri, Nov 15, 2019 at 11:04 AM Jun Rao wrote: > Hi, Satish and Harsha, > > The following is a more detailed high level feedback for the KIP. Overall, > the KIP seems useful. The challenge is how to design it such that it’s > general enough to support different ways of implementing this feature

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-18 Thread Satish Duggana
Hi Jun, Thanks for your detailed review and comments. >40. Local segment metadata storage: The KIP makes the assumption that the metadata for the archived log segments are cached locally in every broker and provides a specific implementation for the local storage in the framework. We probably

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-15 Thread Jun Rao
Hi, Satish and Harsha, The following is a more detailed high level feedback for the KIP. Overall, the KIP seems useful. The challenge is how to design it such that it’s general enough to support different ways of implementing this feature and support existing features. 40. Local segment metadata

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-08 Thread Tom Bentley
Thanks for those insights Ying. On Thu, Nov 7, 2019 at 9:26 PM Ying Zheng wrote: > > > > > > > > Thanks, I missed that point. However, there's still a point at which the > > consumer fetches start getting served from remote storage (even if that > > point isn't as soon as the local log

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-07 Thread Ying Zheng
> > > > Thanks, I missed that point. However, there's still a point at which the > consumer fetches start getting served from remote storage (even if that > point isn't as soon as the local log retention time/size). This represents > a kind of performance cliff edge and what I'm really interested

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-07 Thread Ying Zheng
On Wed, Nov 6, 2019 at 6:28 PM Tom Bentley wrote: > Hi Ying, > > Because only inactive segments can be shipped to remote storage, to be able > > to ship log data as soon > > as possible, we will roll log segment very fast (e.g. every half hour). > > > > So that means a consumer which gets behind

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Satish Duggana
>>Depends on the implementation, the data of one segment may not necessary be stored in a single file. There could be a maximum object / chunk / file size restriction on the remote storage. So, one Kafka segment could be saved in multiple chunks in remote storage. >Having one local segment can be

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Tom Bentley
Hi Satish, >So that means a consumer which gets behind by half an hour will find its > reads being served from remote storage. And, if I understand the proposed > algorithm, each such consumer fetch request could result in a separate > fetch request from the remote storage. I.e. there's no

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Satish Duggana
Hi Tom,' Sorry, I missed the other question. >(Actually the doc for RemoteStorageManager.read() says "It will read at least one batch, if the 1st batch size is larger than maxBytes.". Does that mean the broker might have to retry with increased maxBytes if the first request fails to read a batch?

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Satish Duggana
>So that means a consumer which gets behind by half an hour will find its reads being served from remote storage. And, if I understand the proposed algorithm, each such consumer fetch request could result in a separate fetch request from the remote storage. I.e. there's no mechanism to amortize

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Tom Bentley
Hi Ying, Because only inactive segments can be shipped to remote storage, to be able > to ship log data as soon > as possible, we will roll log segment very fast (e.g. every half hour). > So that means a consumer which gets behind by half an hour will find its reads being served from remote

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Satish Duggana
>Depends on the implementation, the data of one segment may not necessary be stored in a single file. There could be a maximum object / chunk / file size restriction on the remote storage. So, one Kafka segment could be saved in multiple chunks in remote storage. Having one local segment can be

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Ying Zheng
On Wed, Nov 6, 2019 at 4:33 PM Ying Zheng wrote: > 21. I am not sure that I understood the need for RemoteLogIndexEntry and > its relationship with RemoteLogSegmentInfo. It seems > that RemoteLogIndexEntry are offset index entries pointing to record > batches inside a segment. That seems to be

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Ying Zheng
21. I am not sure that I understood the need for RemoteLogIndexEntry and its relationship with RemoteLogSegmentInfo. It seems that RemoteLogIndexEntry are offset index entries pointing to record batches inside a segment. That seems to be the same as the .index file? We do not assume the how the

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-06 Thread Satish Duggana
Hi Jun, >21. Could you elaborate a bit why the positions in remote segment is different from the local one? I thought that they are identical copies. They may not always be the same. Let me take an example here. If remote storage is enabled with encryption then those local positions may not be

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-04 Thread Jun Rao
Hi, Satish, Thanks for the response. 21. Could you elaborate a bit why the positions in remote segment is different from the local one? I thought that they are identical copies. Jun On Fri, Nov 1, 2019 at 4:26 AM Satish Duggana wrote: > Hi Jun, > Thanks for looking into the updated KIP and

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-04 Thread Harsha Chintalapani
Hi Jun, Can you please take a look at Satish's reply. Let us know if that answers your question. I would like to get yours and the rest of the community thoughts on the general direction we are going as we continue to make progress. Thanks, Harsha On Fri, Nov 1, 2019 at 3:06 AM Satish

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-11-01 Thread Satish Duggana
Hi Jun, Thanks for looking into the updated KIP and clarifying our earlier queries. >20. It's fine to keep the HDFS binding temporarily in the PR. We just need to remove it before it's merged to trunk. As Victor mentioned, we can provide a reference implementation based on a mocked version of

Re: [DISCUSS] KIP-405: Kafka Tiered Storage

2019-10-31 Thread Jun Rao
Hi, Harsha, I am still looking at the KIP and the PR. A couple of quick comments/questions. 20. It's fine to keep the HDFS binding temporarily in the PR. We just need to remove it before it's merged to trunk. As Victor mentioned, we can provide a reference implementation based on a mocked

  1   2   >