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 FileType.
That is a good point. We can have suggested changes. 5014. There are some TODO sections in the KIP. Would these be filled up in future iterations? Right. 5015. Under "Topic deletion lifecycle", I'm trying to understand why do we need delete_partition_marked as well as the delete_partition_started messages. I couldn't spot a drawback if supposing we simplified the design such that the controller would only write delete_partition_started message, and RemoteLogCleaner (RLC) instance picks it up for processing. What am I Missing? Having delete_partition_marked event does not add any complexity but it gives audit of the source of the respective action. imho, removing this does not make it simpler. 5016. Under "Topic deletion lifecycle", step (4) is mentioned as "RLC gets all the remote log segments for the partition and each of these remote log segments is deleted with the next steps.". Since the RLC instance runs on each tier topic partition leader, how does the RLC then get the list of remote log segments to be deleted? It will be useful to add that detail to the KIP. Sure, we will address that in the KIP. 5017. Under "Public Interfaces -> Configs", there is a line mentioning "We will support flipping remote.log.storage.enable in next versions." It will be useful to mention this in the "Future Work" section of the KIP too. That makes sense. Will add that in future work items. 5018. The KIP introduces a number of configuration parameters. It will be useful to mention in the KIP if the user should assume these as static configuration in the server.properties file, or dynamic configuration which can be modified without restarting the broker. As discussed earlier, we will update with the config types. 5019. Maybe this is planned as a future update to the KIP, but I thought I'd mention it here. Could you please add details to the KIP on why RocksDB was chosen as the default cache implementation of RLMM, and how it is going to be used? Were alternatives compared/considered? For example, it would be useful to explain/evaluate the following: 1) debuggability of the RocksDB JNI interface, 2) performance, 3) portability across platforms and 4) interface parity of RocksDB’s JNI api with it's underlying C/C++ api. RocksDB is widely used in Kafka Streams. We were evaluating RocksDB and a custom file store. Custom file store adds lot of complexity in maintaining the files and compacting them etc, RocksDB already provides the required features and it is simple to use. We are working on RocksDB implementation with a couple of approaches and we will update the results once we are done. 5020. Following up on (5019), for the RocksDB cache, it will be useful to explain the relationship/mapping between the following in the KIP: 1) # of tiered partitions, 2) # of partitions of metadata topic __remote_log_metadata and 3) # of RocksDB instances. i.e. is the plan to have a RocksDB instance per tiered partition, or per metadata topic partition, or just 1 for per broker? We are exploring of having not more than 2 instances per broker. 5021. I was looking at the implementation prototype (PR link: https://github.com/apache/kafka/pull/7561). It seems that a boolean attribute is being introduced into the Log layer to check if remote log capability is enabled. While the boolean footprint is small at the moment, this can easily grow in the future and become harder to test/maintain, considering that the Log layer is already pretty complex. We should start thinking about how to manage such changes to the Log layer (for the purpose of improved testability, better separation of concerns and readability). One proposal I have is to take a step back and define a higher level Log interface. Then, the Broker code can be changed to use this interface. It can be changed such that only a handle to the interface is exposed to other components (such as LogCleaner, ReplicaManager etc.) and not the underlying Log object. This approach keeps the user of the Log layer agnostic of the whereabouts of the data. Underneath the interface, the implementing classes can completely separate local log capabilities from the remote log. For example, the Log class can be simplified to only manage logic surrounding local log segments and metadata. Additionally, a wrapper class can be provided (implementing the higher level Log interface) which will contain any/all logic surrounding tiered data. The wrapper class will wrap around an instance of the Log class delegating the local log logic to it. Finally, a handle to the wrapper class can be exposed to the other components wherever they need a handle to the higher level Log interface. It is still a draft version and we can discuss code level changes in the PR after it is made ready for review. On Wed, Oct 28, 2020 at 6:27 AM Kowshik Prakasam <kpraka...@confluent.io> wrote: > > 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 duplication, I'd suggest we can instead have a FileType enum and > a common get API based on the FileType. > > 5013. There are some references to the Google doc in the KIP. I wasn't sure > if the Google doc is expected to be in sync with the contents of the wiki. > Going forward, it seems easier if just the KIP is maintained as the source > of truth. In this regard, could you please move all the references to the > Google doc, maybe to a separate References section at the bottom of the KIP? > > 5014. There are some TODO sections in the KIP. Would these be filled up in > future iterations? > > 5015. Under "Topic deletion lifecycle", I'm trying to understand why do we > need delete_partition_marked as well as the delete_partition_started > messages. I couldn't spot a drawback if supposing we simplified the design > such that the controller would only write delete_partition_started message, > and RemoteLogCleaner (RLC) instance picks it up for processing. What am I > missing? > > 5016. Under "Topic deletion lifecycle", step (4) is mentioned as "RLC gets > all the remote log segments for the partition and each of these remote log > segments is deleted with the next steps.". Since the RLC instance runs on > each tier topic partition leader, how does the RLC then get the list of > remote log segments to be deleted? It will be useful to add that detail to > the KIP. > > 5017. Under "Public Interfaces -> Configs", there is a line mentioning "We > will support flipping remote.log.storage.enable in next versions." It will > be useful to mention this in the "Future Work" section of the KIP too. > > 5018. The KIP introduces a number of configuration parameters. It will be > useful to mention in the KIP if the user should assume these as static > configuration in the server.properties file, or dynamic configuration which > can be modified without restarting the broker. > > 5019. Maybe this is planned as a future update to the KIP, but I thought > I'd mention it here. Could you please add details to the KIP on why RocksDB > was chosen as the default cache implementation of RLMM, and how it is going > to be used? Were alternatives compared/considered? For example, it would be > useful to explain/evaluate the following: 1) debuggability of the RocksDB > JNI interface, 2) performance, 3) portability across platforms and 4) > interface parity of RocksDB’s JNI api with it's underlying C/C++ api. > > 5020. Following up on (5019), for the RocksDB cache, it will be useful to > explain the relationship/mapping between the following in the KIP: 1) # of > tiered partitions, 2) # of partitions of metadata topic > __remote_log_metadata and 3) # of RocksDB instances. i.e. is the plan to > have a RocksDB instance per tiered partition, or per metadata topic > partition, or just 1 for per broker? > > 5021. I was looking at the implementation prototype (PR link: > https://github.com/apache/kafka/pull/7561). It seems that a boolean > attribute is being introduced into the Log layer to check if remote log > capability is enabled. While the boolean footprint is small at the moment, > this can easily grow in the future and become harder to > test/maintain, considering that the Log layer is already pretty complex. We > should start thinking about how to manage such changes to the Log layer > (for the purpose of improved testability, better separation of concerns and > readability). One proposal I have is to take a step back and define a > higher level Log interface. Then, the Broker code can be changed to use > this interface. It can be changed such that only a handle to the interface > is exposed to other components (such as LogCleaner, ReplicaManager etc.) > and not the underlying Log object. This approach keeps the user of the Log > layer agnostic of the whereabouts of the data. Underneath the interface, > the implementing classes can completely separate local log capabilities > from the remote log. For example, the Log class can be simplified to only > manage logic surrounding local log segments and metadata. Additionally, a > wrapper class can be provided (implementing the higher level Log interface) > which will contain any/all logic surrounding tiered data. The wrapper > class will wrap around an instance of the Log class delegating the local > log logic to it. Finally, a handle to the wrapper class can be exposed to > the other components wherever they need a handle to the higher level Log > interface. > > > Cheers, > Kowshik > > On Mon, Oct 26, 2020 at 9:52 PM Satish Duggana <satish.dugg...@gmail.com> > wrote: > > > 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 <satish.dugg...@gmail.com> > > wrote: > > > > > > 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 we are looking at: > > > (a) Regular retention for data in tiered storage as per configured ` > > > retention.ms` / `retention.bytes`. > > > (b) Local retention for data in local storage as per configured ` > > > local.log.retention.ms` / `local.log.retention.bytes` > > > (c) Possibly regular retention for data in local storage, if the > > tiering > > > task is lagging or for data that is below the log start offset. > > > > > > Local log retention is done by the existing log cleanup tasks. These > > > are not done for segments that are not yet copied to remote storage. > > > Remote log cleanup is done by the leader partition’s RLMTask. > > > > > > 2. When does a segment become eligible to be tiered? Is it as soon as the > > > segment is rolled and the end offset is less than the last stable offset > > as > > > mentioned in the KIP? I wonder if we need to consider other parameters > > too, > > > like the highwatermark so that we are guaranteed that what we are tiering > > > has been committed to the log and accepted by the ISR. > > > > > > AFAIK, last stable offset is always <= highwatermark. This will make > > > sure we are always tiering the message segments which have been > > > accepted by ISR and transactionally completed. > > > > > > > > > 3. The section on "Follower Fetch Scenarios" is useful but is a bit > > > difficult to parse at the moment. It would be useful to summarize the > > > changes we need in the ReplicaFetcher. > > > > > > It may become difficult for users to read/follow if we add code changes > > here. > > > > > > 4. Related to the above, it's a bit unclear how we are planning on > > > restoring the producer state for a new replica. Could you expand on that? > > > > > > It is mentioned in the KIP BuildingRemoteLogAuxState is introduced to > > > build the state like leader epoch sequence and producer snapshots > > > before it starts fetching the data from the leader. We will make it > > > clear in the KIP. > > > > > > > > > 5. Similarly, it would be worth summarizing the behavior on unclean > > leader > > > election. There are several scenarios to consider here: data loss from > > > local log, data loss from remote log, data loss from metadata topic, etc. > > > It's worth describing these in detail. > > > > > > We mentioned the cases about unclean leader election in the follower > > > fetch scenarios. > > > If there are errors while fetching data from remote store or metadata > > > store, it will work the same way as it works with local log. It > > > returns the error back to the caller. Please let us know if I am > > > missing your point here. > > > > > > > > > 7. For a READ_COMMITTED FetchRequest, how do we retrieve and return the > > > aborted transaction metadata? > > > > > > When a fetch for a remote log is accessed, we will fetch aborted > > > transactions along with the segment if it is not found in the local > > > index cache. This includes the case of transaction index not existing > > > in the remote log segment. That means, the cache entry can be empty or > > > have a list of aborted transactions. > > > > > > > > > 8. The `LogSegmentData` class assumes that we have a log segment, offset > > > index, time index, transaction index, producer snapshot and leader epoch > > > index. How do we deal with cases where we do not have one or more of > > these? > > > For example, we may not have a transaction index or producer snapshot > > for a > > > particular segment. The former is optional, and the latter is only kept > > for > > > up to the 3 latest segments. > > > > > > This is a good point, we discussed this in the last meeting. > > > Transaction index is optional and we will copy them only if it exists. > > > We want to keep all the producer snapshots at each log segment rolling > > > and they can be removed if the log copying is successful and it still > > > maintains the existing latest 3 segments, We only delete the producer > > > snapshots which have been copied to remote log segments on leader. > > > Follower will keep the log segments beyond the segments which have not > > > been copied to remote storage. We will update the KIP with these > > > details. > > > > > > Thanks, > > > Satish. > > > > > > On Thu, Sep 17, 2020 at 1:47 AM Dhruvil Shah <dhru...@confluent.io> > > wrote: > > > > > > > > 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 storage as per configured ` > > > > retention.ms` / `retention.bytes`. > > > > (b) Local retention for data in local storage as per configured ` > > > > local.log.retention.ms` / `local.log.retention.bytes` > > > > (c) Possibly regular retention for data in local storage, if the > > tiering > > > > task is lagging or for data that is below the log start offset. > > > > > > > > 2. When does a segment become eligible to be tiered? Is it as soon as > > the > > > > segment is rolled and the end offset is less than the last stable > > offset as > > > > mentioned in the KIP? I wonder if we need to consider other parameters > > too, > > > > like the highwatermark so that we are guaranteed that what we are > > tiering > > > > has been committed to the log and accepted by the ISR. > > > > > > > > 3. The section on "Follower Fetch Scenarios" is useful but is a bit > > > > difficult to parse at the moment. It would be useful to summarize the > > > > changes we need in the ReplicaFetcher. > > > > > > > > 4. Related to the above, it's a bit unclear how we are planning on > > > > restoring the producer state for a new replica. Could you expand on > > that? > > > > > > > > 5. Similarly, it would be worth summarizing the behavior on unclean > > leader > > > > election. There are several scenarios to consider here: data loss from > > > > local log, data loss from remote log, data loss from metadata topic, > > etc. > > > > It's worth describing these in detail. > > > > > > > > 6. It would be useful to add details about how we plan on using > > RocksDB in > > > > the default implementation of `RemoteLogMetadataManager`. > > > > > > > > 7. For a READ_COMMITTED FetchRequest, how do we retrieve and return the > > > > aborted transaction metadata? > > > > > > > > 8. The `LogSegmentData` class assumes that we have a log segment, > > offset > > > > index, time index, transaction index, producer snapshot and leader > > epoch > > > > index. How do we deal with cases where we do not have one or more of > > these? > > > > For example, we may not have a transaction index or producer snapshot > > for a > > > > particular segment. The former is optional, and the latter is only > > kept for > > > > up to the 3 latest segments. > > > > > > > > Thanks, > > > > Dhruvil > > > > > > > > On Mon, Sep 7, 2020 at 6:54 PM Harsha Ch <harsha...@gmail.com> wrote: > > > > > > > > > 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 < > > satish.dugg...@gmail.com > > > > > > wrote: > > > > > > > > > > > > > > > > > > > > > > > > > > > > > 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 partition by using RLMM". How is this done? > > > > > > > > > > > > > > > > > > > > > > > > It uses RLMM#listSegments() returns all the segments for the given > > topic > > > > > > partition. > > > > > > > > > > > > > > > > > > > > > > > > 600.2 "If the delete option is enabled then the leader will stop > > RLM task > > > > > > and stop processing and it sets all the remote log segment > > metadata of > > > > > > that partition with a delete marker and publishes them to RLMM." We > > > > > > discussed this earlier. When a topic is being deleted, there may > > not be a > > > > > > leader for the deleted partition. > > > > > > > > > > > > > > > > > > > > > > > > This is a good point. As suggested in the meeting, we will add a > > separate > > > > > > section for topic/partition deletion lifecycle and this scenario > > will be > > > > > > addressed. > > > > > > > > > > > > > > > > > > > > > > > > 601. Unclean leader election > > > > > > 601.1 Scenario 1: new empty follower > > > > > > After step 1, the follower restores up to offset 3. So why does it > > have > > > > > > LE-2 <https://issues.apache.org/jira/browse/LE-2> at offset 5? > > > > > > > > > > > > > > > > > > > > > > > > Nice catch. It was showing the leader epoch fetched from the remote > > > > > > storage. It should be shown with the truncated till offset 3. > > Updated the > > > > > > KIP. > > > > > > > > > > > > > > > > > > > > > > > > 601.2 senario 5: After Step 3, leader A has inconsistent data > > between its > > > > > > local and the tiered data. For example. offset 3 has msg 3 LE-0 > > <https://issues.apache.org/jira/browse/LE-0> locally, > > > > > > but msg 5 LE-1 <https://issues.apache.org/jira/browse/LE-1> in > > the remote store. While it's ok for the unclean leader > > > > > > to lose data, it should still return consistent data, whether it's > > from > > > > > > the local or the remote store. > > > > > > > > > > > > > > > > > > > > > > > > There is no inconsistency here as LE-0 > > <https://issues.apache.org/jira/browse/LE-0> offsets are [0, 4] and LE-2 > > <https://issues.apache.org/jira/browse/LE-2>: > > > > > > [5, ]. It will always get the right records for the given offset > > and > > > > > > leader epoch. In case of remote, RSM is invoked to get the remote > > log > > > > > > segment that contains the given offset with the leader epoch. > > > > > > > > > > > > > > > > > > > > > > > > 601.4 It seems that retention is based on > > > > > > listRemoteLogSegments(TopicPartition topicPartition, long > > leaderEpoch). > > > > > > When there is an unclean leader election, it's possible for the new > > > > > leader > > > > > > to not to include certain epochs in its epoch cache. How are remote > > > > > > segments associated with those epochs being cleaned? > > > > > > > > > > > > > > > > > > > > > > > > That is a good point. This leader will also cleanup the epochs > > earlier to > > > > > > its start leader epoch and delete those segments. It gets the > > earliest > > > > > > epoch for a partition and starts deleting segments from that leader > > > > > epoch. > > > > > > We need one more API in RLMM to get the earliest leader epoch. > > > > > > > > > > > > > > > > > > > > > > > > 601.5 The KIP discusses the handling of unclean leader elections > > for user > > > > > > topics. What about unclean leader elections on > > > > > > __remote_log_segment_metadata? > > > > > > This is the same as other system topics like consumer_offsets, > > > > > > __transaction_state topics. As discussed in the meeting, we will > > add the > > > > > > behavior of __remote_log_segment_metadata topic’s unclean leader > > > > > > truncation. > > > > > > > > > > > > > > > > > > > > > > > > 602. It would be useful to clarify the limitations in the initial > > > > > release. > > > > > > The KIP mentions not supporting compacted topics. What about JBOD > > and > > > > > > changing the configuration of a topic from delete to compact after > > > > > remote. > > > > > > log. storage. enable ( http://remote.log.storage.enable/ ) is > > enabled? > > > > > > > > > > > > > > > > > > > > > > > > This was updated in the KIP earlier. > > > > > > > > > > > > > > > > > > > > > > > > 603. RLM leader tasks: > > > > > > 603.1"It checks for rolled over LogSegments (which have the last > > message > > > > > > offset less than last stable offset of that topic partition) and > > copies > > > > > > them along with their offset/time/transaction indexes and leader > > epoch > > > > > > cache to the remote tier." It needs to copy the producer snapshot > > too. > > > > > > > > > > > > > > > > > > > > > > > > Right. It copies producer snapshots too as mentioned in > > LogSegmentData. > > > > > > > > > > > > > > > > > > > > > > > > 603.2 "Local logs are not cleaned up till those segments are copied > > > > > > successfully to remote even though their retention time/size is > > reached" > > > > > > This seems weird. If the tiering stops because the remote store is > > not > > > > > > available, we don't want the local data to grow forever. > > > > > > > > > > > > > > > > > > > > > > > > It was clarified in the discussion that the comment was more about > > the > > > > > > local storage goes beyond the log.retention. The above statement > > is about > > > > > > local.log.retention but not for the complete log.retention. When it > > > > > > reaches the log.retention then it will delete the local logs even > > though > > > > > > those are not copied to remote storage. > > > > > > > > > > > > > > > > > > > > > > > > 604. "RLM maintains a bounded cache(possibly LRU) of the index > > files of > > > > > > remote log segments to avoid multiple index fetches from the remote > > > > > > storage. These indexes can be used in the same way as local segment > > > > > > indexes are used." Could you provide more details on this? Are the > > > > > indexes > > > > > > cached in memory or on disk? If on disk, where are they stored? > > Are the > > > > > > cached indexes bound by a certain size? > > > > > > > > > > > > > > > > > > > > > > > > These are cached on disk and stored in log.dir with a name > > > > > > “__remote_log_index_cache”. They are bound by the total size. This > > will > > > > > be > > > > > > exposed as a user configuration, > > > > > > > > > > > > > > > > > > > > > > > > 605. BuildingRemoteLogAux > > > > > > 605.1 In this section, two options are listed. Which one is chosen? > > > > > > Option-2, updated the KIP. > > > > > > > > > > > > > > > > > > > > > > > > 605.2 In option 2, it says "Build the local leader epoch cache by > > cutting > > > > > > the leader epoch sequence received from remote storage to [LSO, > > ELO]. > > > > > (LSO > > > > > > > > > > > > = log start offset)." We need to do the same thing for the producer > > > > > > snapshot. However, it's hard to cut the producer snapshot to an > > earlier > > > > > > offset. Another option is to simply take the lastOffset from the > > remote > > > > > > segment and use that as the starting fetch offset in the follower. > > This > > > > > > avoids the need for cutting. > > > > > > > > > > > > > > > > > > > > > > > > Right, this was mentioned in the “transactional support” section > > about > > > > > > adding these details. > > > > > > > > > > > > > > > > > > > > > > > > 606. ListOffsets: Since we need a version bump, could you document > > it > > > > > > under a protocol change section? > > > > > > > > > > > > > > > > > > > > > > > > Sure, we will update the KIP. > > > > > > > > > > > > > > > > > > > > > > > > 607. "LogStartOffset of a topic can point to either of local > > segment or > > > > > > remote segment but it is initialised and maintained in the Log > > class like > > > > > > now. This is already maintained in `Log` class while loading the > > logs and > > > > > > it can also be fetched from RemoteLogMetadataManager." What will > > happen > > > > > to > > > > > > the existing logic (e.g. log recovery) that currently depends on > > > > > > logStartOffset but assumes it's local? > > > > > > > > > > > > > > > > > > > > > > > > They use a field called localLogStartOffset which is the local log > > start > > > > > > offset.. > > > > > > > > > > > > > > > > > > > > > > > > 608. Handle expired remote segment: How does it pick up new > > > > > logStartOffset > > > > > > from deleteRecords? > > > > > > > > > > > > > > > > > > > > > > > > Good point. This was not addressed in the KIP. Will update the KIP > > on how > > > > > > the RLM task handles this scenario. > > > > > > > > > > > > > > > > > > > > > > > > 609. RLMM message format: > > > > > > 609.1 It includes both MaxTimestamp and EventTimestamp. Where does > > it get > > > > > > both since the message in the log only contains one timestamp? > > > > > > > > > > > > > > > > > > > > > > > > `EventTimeStamp` is the timestamp at which that segment metadata > > event is > > > > > > generated. This is more for audits. > > > > > > > > > > > > > > > > > > > > > > > > 609.2 If we change just the state (e.g. to DELETE_STARTED), it > > seems it's > > > > > > wasteful to have to include all other fields not changed. > > > > > > > > > > > > > > > > > > > > > > > > This is a good point. We thought about incremental updates. But we > > want > > > > > to > > > > > > make sure all the events are in the expected order and take action > > based > > > > > > on the latest event. Will think through the approaches in detail > > and > > > > > > update here. > > > > > > > > > > > > > > > > > > > > > > > > 609.3 Could you document which process makes the following > > transitions > > > > > > DELETE_MARKED, DELETE_STARTED, DELETE_FINISHED? > > > > > > > > > > > > > > > > > > > > > > > > Okay, will document more details. > > > > > > > > > > > > > > > > > > > > > > > > 610. remote.log.reader.max.pending.tasks: "Maximum remote log > > reader > > > > > > thread pool task queue size. If the task queue is full, broker > > will stop > > > > > > reading remote log segments." What does the broker do if the queue > > is > > > > > > full? > > > > > > > > > > > > > > > > > > > > > > > > It returns an error for this topic partition. > > > > > > > > > > > > > > > > > > > > > > > > 611. What do we return if the request offset/epoch doesn't exist > > in the > > > > > > following API? > > > > > > RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition > > > > > > topicPartition, long offset, int epochForOffset) > > > > > > > > > > > > > > > > > > > > > > > > This returns null. But we prefer to update the return type as > > Optional > > > > > and > > > > > > return Empty if that does not exist. > > > > > > > > > > > > > > > > > > > > > > > > Thanks, > > > > > > Satish. > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Sep 1, 2020 at 9:45 AM Jun Rao < jun@ confluent. io ( > > > > > > j...@confluent.io ) > wrote: > > > > > > > > > > > > > > > > > >> > > > > > >> > > > > > >> 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 by using RLMM". How is this done? 600.2 "If the > > delete > > > > > >> option is enabled then the leader will stop RLM task and stop > > processing > > > > > >> and it sets all the remote log segment metadata of that partition > > with a > > > > > >> delete marker and publishes them to RLMM." We discussed this > > earlier. > > > > > When > > > > > >> a topic is being deleted, there may not be a leader for the > > deleted > > > > > >> partition. > > > > > >> > > > > > >> > > > > > >> > > > > > >> 601. Unclean leader election > > > > > >> 601.1 Scenario 1: new empty follower > > > > > >> After step 1, the follower restores up to offset 3. So why does > > it have > > > > > >> LE-2 <https://issues.apache.org/jira/browse/LE-2> at offset 5? > > > > > >> 601.2 senario 5: After Step 3, leader A has inconsistent data > > between > > > > > its > > > > > >> local and the tiered data. For example. offset 3 has msg 3 LE-0 > > <https://issues.apache.org/jira/browse/LE-0> locally, > > > > > >> but msg 5 LE-1 <https://issues.apache.org/jira/browse/LE-1> in > > the remote store. While it's ok for the unclean leader > > > > > >> to lose data, it should still return consistent data, whether > > it's from > > > > > >> the local or the remote store. > > > > > >> 601.3 The follower picks up log start offset using the following > > api. > > > > > >> Suppose that we have 3 remote segments (LE, SegmentStartOffset) > > as (2, > > > > > >> 10), > > > > > >> (3, 20) and (7, 15) due to an unclean leader election. Using the > > > > > following > > > > > >> api will cause logStartOffset to go backward from 20 to 15. How > > do we > > > > > >> prevent that? > > > > > >> earliestLogOffset(TopicPartition topicPartition, int leaderEpoch) > > 601.4 > > > > > It > > > > > >> seems that retention is based on > > > > > >> listRemoteLogSegments(TopicPartition topicPartition, long > > leaderEpoch). > > > > > >> When there is an unclean leader election, it's possible for the > > new > > > > > leader > > > > > >> to not to include certain epochs in its epoch cache. How are > > remote > > > > > >> segments associated with those epochs being cleaned? 601.5 The KIP > > > > > >> discusses the handling of unclean leader elections for user > > topics. What > > > > > >> about unclean leader elections on > > > > > >> __remote_log_segment_metadata? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 602. It would be useful to clarify the limitations in the initial > > > > > release. > > > > > >> The KIP mentions not supporting compacted topics. What about JBOD > > and > > > > > >> changing the configuration of a topic from delete to compact after > > > > > remote. > > > > > >> log. storage. enable ( http://remote.log.storage.enable/ ) is > > enabled? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 603. RLM leader tasks: > > > > > >> 603.1"It checks for rolled over LogSegments (which have the last > > message > > > > > >> offset less than last stable offset of that topic partition) and > > copies > > > > > >> them along with their offset/time/transaction indexes and leader > > epoch > > > > > >> cache to the remote tier." It needs to copy the producer snapshot > > too. > > > > > >> 603.2 "Local logs are not cleaned up till those segments are > > copied > > > > > >> successfully to remote even though their retention time/size is > > reached" > > > > > >> This seems weird. If the tiering stops because the remote store > > is not > > > > > >> available, we don't want the local data to grow forever. > > > > > >> > > > > > >> > > > > > >> > > > > > >> 604. "RLM maintains a bounded cache(possibly LRU) of the index > > files of > > > > > >> remote log segments to avoid multiple index fetches from the > > remote > > > > > >> storage. These indexes can be used in the same way as local > > segment > > > > > >> indexes are used." Could you provide more details on this? Are the > > > > > indexes > > > > > >> cached in memory or on disk? If on disk, where are they stored? > > Are the > > > > > >> cached indexes bound by a certain size? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 605. BuildingRemoteLogAux > > > > > >> 605.1 In this section, two options are listed. Which one is > > chosen? > > > > > 605.2 > > > > > >> In option 2, it says "Build the local leader epoch cache by > > cutting the > > > > > >> leader epoch sequence received from remote storage to [LSO, ELO]. > > (LSO > > > > > >> = log start offset)." We need to do the same thing for the > > producer > > > > > >> snapshot. However, it's hard to cut the producer snapshot to an > > earlier > > > > > >> offset. Another option is to simply take the lastOffset from the > > remote > > > > > >> segment and use that as the starting fetch offset in the > > follower. This > > > > > >> avoids the need for cutting. > > > > > >> > > > > > >> > > > > > >> > > > > > >> 606. ListOffsets: Since we need a version bump, could you > > document it > > > > > >> under a protocol change section? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 607. "LogStartOffset of a topic can point to either of local > > segment or > > > > > >> remote segment but it is initialised and maintained in the Log > > class > > > > > like > > > > > >> now. This is already maintained in `Log` class while loading the > > logs > > > > > and > > > > > >> it can also be fetched from RemoteLogMetadataManager." What will > > happen > > > > > to > > > > > >> the existing logic (e.g. log recovery) that currently depends on > > > > > >> logStartOffset but assumes it's local? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 608. Handle expired remote segment: How does it pick up new > > > > > logStartOffset > > > > > >> from deleteRecords? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 609. RLMM message format: > > > > > >> 609.1 It includes both MaxTimestamp and EventTimestamp. Where > > does it > > > > > get > > > > > >> both since the message in the log only contains one timestamp? > > 609.2 If > > > > > we > > > > > >> change just the state (e.g. to DELETE_STARTED), it seems it's > > wasteful > > > > > to > > > > > >> have to include all other fields not changed. 609.3 Could you > > document > > > > > >> which process makes the following transitions DELETE_MARKED, > > > > > >> DELETE_STARTED, DELETE_FINISHED? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 610. remote.log.reader.max.pending.tasks: "Maximum remote log > > reader > > > > > >> thread pool task queue size. If the task queue is full, broker > > will stop > > > > > >> reading remote log segments." What does the broker do if the > > queue is > > > > > >> full? > > > > > >> > > > > > >> > > > > > >> > > > > > >> 611. What do we return if the request offset/epoch doesn't exist > > in the > > > > > >> following API? > > > > > >> RemoteLogSegmentMetadata remoteLogSegmentMetadata(TopicPartition > > > > > >> topicPartition, long offset, int epochForOffset) > > > > > >> > > > > > >> > > > > > >> > > > > > >> Jun > > > > > >> > > > > > >> > > > > > >> > > > > > >> On Mon, Aug 31, 2020 at 11:19 AM Satish Duggana < satish. duggana@ > > > > > gmail. com > > > > > >> ( satish.dugg...@gmail.com ) > wrote: > > > > > >> > > > > > >> > > > > > >>> > > > > > >>> > > > > > >>> 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, > > > > > >>> Satish. > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> On Thu, Aug 27, 2020 at 12:42 AM Harsha Ch < harsha. ch@ gmail. > > com ( > > > > > >>> harsha...@gmail.com ) > wrote: > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>> > > > > > >>>> Updated the KIP with Meeting Notes section > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-MeetingNotes > > > > > >>> ( > > > > > >>> > > > > > > > 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 < jun@ confluent. io ( > > > > > >>>> j...@confluent.io ) > wrote: > > > > > >>>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> 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 > > > > > >>> ( > > > > > >>> > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals > > > > > >>> ) > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> ? > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> Jun > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> On Tue, Aug 25, 2020 at 11:12 AM Harsha Chintalapani < kafka@ > > > > > harsha. io ( > > > > > >>>>> ka...@harsha.io ) > wrote: > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> Thanks everyone for attending the meeting today. > > > > > >>>>>> Here is the recording > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> https:/ / drive. google. com/ file/ d/ > > > > > 14PRM7U0OopOOrJR197VlqvRX5SXNtmKj/ view?usp=sharing > > > > > >>> ( > > > > > >>> > > > > > > > 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 the > > KIP > > > > > >>>>>> 3. We will update the KIP with details of different cases > > > > > >>>>>> 4. Test plan will be captured in a doc and will add to the KIP > > > > > >>>>>> 5. Add a section "Limitations" to capture the capabilities > > that > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> will > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> be > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> introduced with this KIP and what will not be covered in this > > KIP. > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> Please add to it I missed anything. Will produce a formal > > meeting > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> notes > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> from next meeting onwards. > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> Thanks, > > > > > >>>>>> Harsha > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> On Mon, Aug 24, 2020 at 9:42 PM, Ying Zheng < yingz@ uber. > > com. > > > > > invalid ( > > > > > >>>>>> yi...@uber.com.invalid ) > wrote: > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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/ ( > > > > > >>>>>>> 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 11:10 AM Harsha Ch < harsha. ch@ > > gmail. > > > > > com ( > > > > > >>>>>>> harsha...@gmail.com ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> wrote: > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> "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 < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> kprakasam@ confluent. io ( kpraka...@confluent.io ) > > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi Harsha, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> The following google doc contains a proposal for temporary > > agenda > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> for > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> the > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> KIP-405 <https://issues.apache.org/jira/browse/KIP-405> < > > https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > sync > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> meeting > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> tomorrow: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / docs. google. com/ document/ d/ ( > > > > > >>>>>>> https://docs.google.com/document/d/ ) > > > > > >>>>>>> 1pqo8X5LU8TpwfC_iqSuVPezhfCfhGkbGN2TqiPA3LBU/edit > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> . > > > > > >>>>>>> Please could you add it to the Google calendar invite? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thank you. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Cheers, > > > > > >>>>>>> Kowshik > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Thu, Aug 20, 2020 at 10:58 AM Harsha Ch < harsha. ch@ > > gmail. > > > > > com ( > > > > > >>>>>>> harsha...@gmail.com ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> wrote: > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi All, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Scheduled a meeting for Tuesday 9am - 10am. I can record and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> upload for > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> community to be able to follow the discussion. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Jun, please add the required folks on confluent side. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Harsha > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Thu, Aug 20, 2020 at 12:33 AM, Alexandre Dupriez < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>>> alexandre.dupriez@ > > > > > >>>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> gmail. com ( http://gmail.com/ ) > wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 < harsha. ch@ > > gmail. com ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>> harsha. > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ch@ gmail. com ( c...@gmail.com ) ) > a écrit : > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 < jun@ confluent. > > io ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> jun@ > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> confluent. io ( http://confluent.io/ ) ) > 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 > > > > > >>>>>>> design/development progress and discussing any open issues to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> accelerate > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this KIP. If so, will every Tuesday (from next week) 9am-10am > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> PT > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> work for you? I can help set up a Zoom meeting, invite > > everyone who > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> might > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> be interested, have it recorded and shared, etc. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Jun > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Tue, Aug 18, 2020 at 11:01 AM Satish Duggana < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> satish. duggana@ gmail. com ( satish. duggana@ gmail. com ( > > > > > >>>>>>> satish.dugg...@gmail.com ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> next-local-offset? last-tiered-offset seems to naturally > > align well > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the definition provided in the KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Both next-local-offset and local-log-start-offset were > > introduced > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> talk > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> about offsets related to local log. We are fine with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> last-tiered-offset > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> too as you suggested. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5002. After leadership is established for a partition, the > > leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> would > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> begin uploading a segment to remote storage. If successful, > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> would write the updated RemoteLogSegmentMetadata to the > > metadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (via > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> RLMM.putRemoteLogSegmentData). However, for defensive > > reasons, it > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> seems > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> useful that before the first time the segment is uploaded by > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> a partition, the leader should ensure to catch up to all the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> metadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> events written so far in the metadata topic for that > > partition (ex: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> previous leader). To achieve this, the leader could start a > > lease > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (using > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> establish_leader metadata event) before commencing tiering, > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wait > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> until > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the event is read back. For example, this seems useful to > > avoid > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> cases > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> where > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> zombie leaders can be active for the same partition. This > > can also > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> prove > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> useful to help avoid making decisions on which segments to be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> uploaded > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> a partition, until the current leader has caught up to a > > complete > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> view > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> of > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> all segments uploaded for the partition so far (otherwise > > this may > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> cause > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> same segment being uploaded twice -- once by the previous > > leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> then > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by the new leader). > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> We allow copying segments to remote storage which may have > > common > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> offsets. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Please go through the KIP to understand the follower fetch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> protocol(1) and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> follower to leader transition(2). > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ ( > > > > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ ( > > > > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5003. There is a natural interleaving between uploading a > > segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> remote > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> store, and, writing a metadata event for the same (via > > > > > >>>>>>> RLMM.putRemoteLogSegmentData). There can be cases where a > > remote > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> uploaded, then the leader fails and a corresponding metadata > > event > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> never > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> gets written. In such cases, the orphaned remote segment has > > to be > > > > > >>>>>>> eventually deleted (since there is no confirmation of the > > upload). > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> To > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> handle this, we could use 2 separate metadata events viz. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> copy_initiated > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and copy_completed, so that copy_initiated events that don't > > have a > > > > > >>>>>>> corresponding copy_completed event can be treated as garbage > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> deleted > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> from the remote object store by the broker. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> We are already updating RMM with RemoteLogSegmentMetadata > > pre and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> post > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> copying of log segments. We had a flag in > > RemoteLogSegmentMetadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> whether > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it is copied or not. But we are making changes in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> RemoteLogSegmentMetadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to introduce a state field in RemoteLogSegmentMetadata which > > will > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> have the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> respective started and finished states. This includes for > > other > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> operations > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> like delete too. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5004. In the default implementation of RLMM (using the > > internal > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> __remote_log_metadata), a separate topic called > > > > > >>>>>>> __remote_segments_to_be_deleted is going to be used just to > > track > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> failures > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in removing remote log segments. A separate topic > > (effectively > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> another > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> metadata stream) introduces some maintenance overhead and > > design > > > > > >>>>>>> complexity. It seems to me that the same can be achieved > > just by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> using > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> just > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the __remote_log_metadata topic with the following steps: 1) > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> writes a delete_initiated metadata event, 2) the leader > > deletes the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and 3) the leader writes a delete_completed metadata event. > > Tiered > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segments > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> that have delete_initiated message and not delete_completed > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> message, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> can > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> considered to be a failure and retried. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Jun suggested in earlier mail to keep this simple . We > > decided not > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to have > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this topic as mentioned in our earlier replies, updated the > > KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> As I > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> mentioned in an earlier comment, we are adding state entries > > for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> delete > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> operations too. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5005. When a Kafka cluster is provisioned for the first time > > with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> KIP-405 <https://issues.apache.org/jira/browse/KIP-405> < > > https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> tiered storage enabled, could you explain in the KIP about > > how the > > > > > >>>>>>> bootstrap for __remote_log_metadata topic will be performed > > in the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> default RLMM implementation? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> __remote_log_segment_metadata topic is created by default > > with the > > > > > >>>>>>> respective topic like partitions/replication-factor etc. Can > > you be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> more > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> specific on what you are looking for? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5008. The system-wide configuration ' remote. log. storage. > > enable > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> http:/ / remote. log. storage. enable/ ( > > > > > http://remote.log.storage.enable/ > > > > > >>>>>>> ) ) ' is used > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> enable tiered storage. Can this be made a topic-level > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> configuration, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> so > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> that the user can enable/disable tiered storage at a topic > > level > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> rather > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> than a system-wide default for an entire Kafka cluster? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Yes, we mentioned in an earlier mail thread that it will be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> supported at > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic level too, updated the KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5009. Whenever a topic with tiered storage enabled is > > deleted, the > > > > > >>>>>>> underlying actions require the topic data to be deleted in > > local > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> store > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> as > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> well as remote store, and eventually the topic metadata > > needs to be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> deleted > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> too. What is the role of the controller in deleting a topic > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it's > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> contents, while the topic has tiered storage enabled? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> When a topic partition is deleted, there will be an event > > for that > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in RLMM > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> for its deletion and the controller considers that topic is > > deleted > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> only > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> when all the remote log segments are also deleted. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5010. RLMM APIs are currently synchronous, for example > > > > > >>>>>>> RLMM.putRemoteLogSegmentData waits until the put operation is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> completed > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the remote metadata store. It may also block until the > > leader has > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> caught > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> up > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to the metadata (not sure). Could we make these apis > > asynchronous > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (ex: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> based on java.util.concurrent.Future) to provide room for > > tapping > > > > > >>>>>>> performance improvements such as non-blocking i/o? 5011. The > > same > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> question > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> as 5009 on sync vs async api for RSM. Have we considered the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> pros/cons of > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> making the RSM apis asynchronous? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Async methods are used to do other tasks while the result is > > not > > > > > >>>>>>> available. In this case, we need to have the result before > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> proceeding to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> take next actions. These APIs are evolving and these can be > > updated > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> as and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> when needed instead of having them as asynchronous now. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> Satish. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Fri, Aug 14, 2020 at 4:30 AM Kowshik Prakasam < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> kprakasam@ confluent. io ( kprakasam@ confluent. io ( > > > > > >>>>>>> kpraka...@confluent.io ) ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 last-tiered-offset be a > > better name > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> than > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> next-local-offset? last-tiered-offset seems to naturally > > align > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> well > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the definition provided in the KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5002. After leadership is established for a partition, the > > leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> would > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> begin uploading a segment to remote storage. If successful, > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> would write the updated RemoteLogSegmentMetadata to the > > metadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (via > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> RLMM.putRemoteLogSegmentData). However, for defensive > > reasons, it > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> seems > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> useful that before the first time the segment is uploaded by > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> a partition, the leader should ensure to catch up to all the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> metadata > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> events written so far in the metadata topic for that > > partition > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (ex: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> previous leader). To achieve this, the leader could start a > > lease > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (using > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> establish_leader metadata event) before commencing tiering, > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wait > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> until > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the event is read back. For example, this seems useful to > > avoid > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> cases > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> where > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> zombie leaders can be active for the same partition. This > > can also > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> prove > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> useful to help avoid making decisions on which segments to be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> uploaded > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> a partition, until the current leader has caught up to a > > complete > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> view > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> of > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> all segments uploaded for the partition so far (otherwise > > this may > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> cause > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> same segment being uploaded twice -- once by the previous > > leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> then > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by the new leader). > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5003. There is a natural interleaving between uploading a > > segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> remote > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> store, and, writing a metadata event for the same (via > > > > > >>>>>>> RLMM.putRemoteLogSegmentData). There can be cases where a > > remote > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> uploaded, then the leader fails and a corresponding metadata > > event > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> never > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> gets written. In such cases, the orphaned remote segment has > > to be > > > > > >>>>>>> eventually deleted (since there is no confirmation of the > > upload). > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> To > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> handle this, we could use 2 separate metadata events viz. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> copy_initiated > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and copy_completed, so that copy_initiated events that don't > > have > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> a > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> corresponding copy_completed event can be treated as garbage > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> deleted > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> from the remote object store by the broker. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5004. In the default implementation of RLMM (using the > > internal > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> __remote_log_metadata), a separate topic called > > > > > >>>>>>> __remote_segments_to_be_deleted is going to be used just to > > track > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> failures > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in removing remote log segments. A separate topic > > (effectively > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> another > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> metadata stream) introduces some maintenance overhead and > > design > > > > > >>>>>>> complexity. It seems to me that the same can be achieved > > just by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> using > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> just > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the __remote_log_metadata topic with the following steps: 1) > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> writes a delete_initiated metadata event, 2) the leader > > deletes > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and 3) the leader writes a delete_completed metadata event. > > Tiered > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segments > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> that have delete_initiated message and not delete_completed > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> message, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> can > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> considered to be a failure and retried. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5005. When a Kafka cluster is provisioned for the first time > > with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> KIP-405 <https://issues.apache.org/jira/browse/KIP-405> < > > https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> tiered storage enabled, could you explain in the KIP about > > how the > > > > > >>>>>>> bootstrap for __remote_log_metadata topic will be performed > > in the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> default RLMM implementation? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5006. I currently do not see details on the KIP on why > > RocksDB was > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> chosen > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> as the default cache implementation, and how it is going to > > be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> used. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Were > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> alternatives compared/considered? For example, it would be > > useful > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> explain/evaulate the following: 1) debuggability of the > > RocksDB > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> JNI > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> interface, 2) performance, 3) portability across platforms > > and 4) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> interface > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> parity of RocksDB’s JNI api with it's underlying C/C++ api. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5007. For the RocksDB cache (the default implementation of > > RLMM), > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> what > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the relationship/mapping between the following: 1) # of > > tiered > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> partitions, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 2) # of partitions of metadata topic __remote_log_metadata > > and 3) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> # > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> of > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> RocksDB instances? i.e. is the plan to have a RocksDB > > instance per > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> tiered > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> partition, or per metadata topic partition, or just 1 for per > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> broker? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5008. The system-wide configuration ' remote. log. storage. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> enable ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> http:/ / remote. log. storage. enable/ ( > > > > > http://remote.log.storage.enable/ > > > > > >>>>>>> ) ) ' is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> used > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> enable tiered storage. Can this be made a topic-level > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> configuration, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> so > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> that the user can enable/disable tiered storage at a topic > > level > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> rather > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> than a system-wide default for an entire Kafka cluster? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5009. Whenever a topic with tiered storage enabled is > > deleted, the > > > > > >>>>>>> underlying actions require the topic data to be deleted in > > local > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> store > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> as > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> well as remote store, and eventually the topic metadata > > needs to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> be > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> deleted > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> too. What is the role of the controller in deleting a topic > > and > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it's > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> contents, while the topic has tiered storage enabled? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5010. RLMM APIs are currently synchronous, for example > > > > > >>>>>>> RLMM.putRemoteLogSegmentData waits until the put operation is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> completed > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the remote metadata store. It may also block until the > > leader has > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> caught > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> up > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> to the metadata (not sure). Could we make these apis > > asynchronous > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (ex: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> based on java.util.concurrent.Future) to provide room for > > tapping > > > > > >>>>>>> performance improvements such as non-blocking i/o? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 5011. The same question as 5009 on sync vs async api for > > RSM. Have > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> we > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> considered the pros/cons of making the RSM apis asynchronous? > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Cheers, > > > > > >>>>>>> Kowshik > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Thu, Aug 6, 2020 at 11:02 AM Satish Duggana < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> satish. duggana@ gmail. com ( satish. duggana@ gmail. com ( > > > > > >>>>>>> satish.dugg...@gmail.com ) ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 remaining part of the KIP coming along? To me, the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> two > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> biggest > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> missing items are (1) more detailed documentation on how all > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> new > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> APIs > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> are being used and (2) metadata format and usage in the > > internal > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> __remote_log_metadata. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> We are working on updating APIs based on the recent > > discussions > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and get > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the perf numbers by plugging in rocksdb as a cache store for > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> RLMM. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> We will update the KIP with the updated APIs and with the > > above > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> requested > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> details in a few days and let you know. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> Satish. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Wed, Aug 5, 2020 at 12:49 AM Jun Rao < jun@ confluent. > > io ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> jun@ > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> confluent. io ( http://confluent.io/ ) ) > wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi, Ying, Satish, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks for the reply. 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. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> How is the remaining part of the KIP coming along? To me, the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> two > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> biggest > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> missing items are (1) more detailed documentation on how all > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> new > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> APIs > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> are being used and (2) metadata format and usage in the > > internal > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> topic > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> __remote_log_metadata. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thanks, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Jun > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Tue, Aug 4, 2020 at 8:32 AM Satish Duggana < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> satish. duggana@ gmail. com ( satish. duggana@ gmail. com ( > > > > > >>>>>>> satish.dugg...@gmail.com ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi Jun, > > > > > >>>>>>> Thanks for your comment, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 ooffset by the > > previous > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> but > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it seems that's not the case. It would be useful to document > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Wiki. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> When a follower becomes a leader it needs to findout the > > offset > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> from > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> which the segments to be copied to remote storage. This is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> found > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> traversing from the the latest leader epoch from leader epoch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> history > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and find the highest offset of a segment with that epoch > > copied > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> into > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> remote storage by using respective RLMM APIs. If it can not > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> find > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> entry then it checks for the previous leader epoch till it > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> finds > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> entry, If there are no entries till the earliest leader epoch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader epoch cache then it starts copying the segments from > > the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> earliest > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> epoch entry’s offset. > > > > > >>>>>>> Added an example in the KIP here[1]. We will update RLMM APIs > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ ( > > > > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Satish. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Tue, Aug 4, 2020 at 9:00 PM Satish Duggana < > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> satish. duggana@ gmail. com ( satish. duggana@ gmail. com ( > > > > > >>>>>>> satish.dugg...@gmail.com ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi Ying, > > > > > >>>>>>> Thanks for your comment. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 ooffset by the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> previous > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> but > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it seems that's not the case. It would be useful to document > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Wiki. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> When a follower becomes a leader it needs to findout the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> offset > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> from > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> which the segments to be copied to remote storage. This is > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> found > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> by > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> traversing from the the latest leader epoch from leader epoch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> history > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> and find the highest offset of a segment with that epoch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> copied > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> into > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> remote storage by using respective RLMM APIs. If it can not > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> find > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> entry then it checks for the previous leader epoch till it > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> finds > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> an > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> entry, If there are no entries till the earliest leader epoch > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader epoch cache then it starts copying the segments from > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> earliest epoch entry’s offset. > > > > > >>>>>>> Added an example in the KIP here[1]. We will update RLMM APIs > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> KIP. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < https:/ / issues. apache. org/ jira/ browse/ KIP-405 > > <https://issues.apache.org/jira/browse/KIP-405> ( > > > > > >>>>>>> https://issues.apache.org/jira/browse/KIP-405 ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> https:/ / cwiki. apache. org/ confluence/ display/ KAFKA/ ( > > > > > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/ ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>> > > > > > KIP-405 <https://issues.apache.org/jira/browse/KIP-405> > > %3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-Followertoleadertransition > > > > > >>> > > > > > >>> > > > > > >>> > > > > > >>>> > > > > > >>>>> > > > > > >>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> ) > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Satish. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Tue, Aug 4, 2020 at 10:28 AM Ying Zheng > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < yingz@ uber. com. invalid ( yingz@ uber. com. invalid ( > > > > > >>>>>>> yi...@uber.com.invalid ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Hi Jun, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Thank you for the comment! The current KIP is not very > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> clear > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> about > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> part. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 1001. The new leader will start archiving from the earliest > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> local > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> segment > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> that is not fully > > > > > >>>>>>> covered by the "valid" remote data. "valid" means the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> (offset, > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> epoch) pair is valid > > > > > >>>>>>> based on the leader-epoch history. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> There are some edge cases where the same offset range (with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> same > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> epoch) can > > > > > >>>>>>> be copied to the remote storage more than once. But this > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> kind > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> of > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> duplication shouldn't be a > > > > > >>>>>>> problem. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Staish is going to explain the details in the KIP with > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> examples. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Fri, Jul 31, 2020 at 2:55 PM Jun Rao < jun@ confluent. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> io ( > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> jun@ confluent. io ( j...@confluent.io ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 ooffset by the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> previous > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> leader, but > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> it seems that's not the case. It would be useful to > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> document > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> this in > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> the > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wiki. > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> Jun > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> On Tue, Jul 28, 2020 at 12:11 PM Ying Zheng > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> < yingz@ uber. com. invalid ( yingz@ uber. com. invalid ( > > > > > >>>>>>> yi...@uber.com.invalid ) ) > > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> wrote: > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>>> 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 > > > > > >>>>>>> > > > > > >>>>>>> > > > > > >>>>>> > > > > > >>>>>> > > > > > >>>>> > > > > > >>>>> > > > > > >>>> > > > > > >>>> > > > > > >>> > > > > > >>> > > > > > >> > > > > > >> > > > > > > > > > > > > > > > > > > > >